diff --git a/cmd/master/main.go b/cmd/master/main.go index ac4084458e..c450fb4615 100644 --- a/cmd/master/main.go +++ b/cmd/master/main.go @@ -12,7 +12,6 @@ import ( "syscall" ) - func main() { var yamlFile string diff --git a/cmd/reader/reader.go b/cmd/reader/reader.go index e878f0aac9..f229b1d461 100644 --- a/cmd/reader/reader.go +++ b/cmd/reader/reader.go @@ -12,7 +12,7 @@ import ( func main() { ctx, _ := context.WithCancel(context.Background()) - var yamlFile string + var yamlFile string flag.StringVar(&yamlFile, "yaml", "", "yaml file") flag.Parse() // flag.Usage() @@ -25,4 +25,3 @@ func main() { pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10) reader.StartQueryNode(ctx, pulsarAddr) } - diff --git a/cmd/storage/benchmark.go b/cmd/storage/benchmark.go index 699ac3763b..2e7a35727d 100644 --- a/cmd/storage/benchmark.go +++ b/cmd/storage/benchmark.go @@ -5,9 +5,9 @@ import ( "crypto/md5" "flag" "fmt" + "github.com/pivotal-golang/bytefmt" "github.com/zilliztech/milvus-distributed/internal/storage" "github.com/zilliztech/milvus-distributed/internal/storage/type" - "github.com/pivotal-golang/bytefmt" "log" "math/rand" "os" @@ -31,14 +31,13 @@ var logFile *os.File var store storagetype.Store var wg sync.WaitGroup - func runSet() { for time.Now().Before(endTime) { num := atomic.AddInt32(&keyNum, 1) key := []byte(fmt.Sprint("key", num)) for ver := 1; ver <= numVersion; ver++ { atomic.AddInt32(&counter, 1) - err := store.PutRow(context.Background(), key, valueData,"empty" ,uint64(ver)) + err := store.PutRow(context.Background(), key, valueData, "empty", uint64(ver)) if err != nil { log.Fatalf("Error setting key %s, %s", key, err.Error()) //atomic.AddInt32(&setCount, -1) @@ -61,7 +60,7 @@ func runBatchSet() { } for ver := 1; ver <= numVersion; ver++ { atomic.AddInt32(&counter, 1) - err := store.PutRows(context.Background(), keys, batchValueData, batchSuffix,versions) + err := store.PutRows(context.Background(), keys, batchValueData, batchSuffix, versions) if err != nil { log.Fatalf("Error setting batch keys %s %s", keys, err.Error()) //atomic.AddInt32(&batchSetCount, -1) @@ -72,7 +71,6 @@ func runBatchSet() { wg.Done() } - func runGet() { for time.Now().Before(endTime) { num := atomic.AddInt32(&counter, 1) @@ -105,8 +103,8 @@ func runBatchGet() { start := end - int32(batchOpSize) keys := totalKeys[start:end] versions := make([]uint64, batchOpSize) - for i, _ := range versions{ - versions[i]= uint64(numVersion) + for i, _ := range versions { + versions[i] = uint64(numVersion) } atomic.AddInt32(&counter, 1) _, err := store.GetRows(context.Background(), keys, versions) @@ -153,8 +151,8 @@ func runBatchDelete() { keys := totalKeys[start:end] atomic.AddInt32(&counter, 1) versions := make([]uint64, batchOpSize) - for i, _ := range versions{ - versions[i]= uint64(numVersion) + for i, _ := range versions { + versions[i] = uint64(numVersion) } err := store.DeleteRows(context.Background(), keys, versions) if err != nil { @@ -239,7 +237,7 @@ func main() { setTime := setFinish.Sub(startTime).Seconds() bps := float64(uint64(counter)*valueSize*uint64(batchOpSize)) / setTime fmt.Fprint(logFile, fmt.Sprintf("Loop %d: BATCH PUT time %.1f secs, batchs = %d, kv pairs = %d, speed = %sB/sec, %.1f operations/sec, %.1f kv/sec.\n", - loop, setTime, counter, counter*int32(batchOpSize), bytefmt.ByteSize(uint64(bps)), float64(counter)/setTime, float64(counter * int32(batchOpSize))/setTime)) + loop, setTime, counter, counter*int32(batchOpSize), bytefmt.ByteSize(uint64(bps)), float64(counter)/setTime, float64(counter*int32(batchOpSize))/setTime)) // Record all test keys //totalKeyCount = keyNum diff --git a/cmd/writer/writer.go b/cmd/writer/writer.go index 09be2d694c..23137d8ee8 100644 --- a/cmd/writer/writer.go +++ b/cmd/writer/writer.go @@ -5,18 +5,18 @@ import ( "flag" "fmt" "github.com/zilliztech/milvus-distributed/internal/conf" - "github.com/zilliztech/milvus-distributed/internal/storage" "github.com/zilliztech/milvus-distributed/internal/msgclient" + "github.com/zilliztech/milvus-distributed/internal/storage" "github.com/zilliztech/milvus-distributed/internal/writer" "log" "strconv" ) func main() { - var yamlFile string + var yamlFile string flag.StringVar(&yamlFile, "yaml", "", "yaml file") flag.Parse() - // flag.Usage() + // flag.Usage() fmt.Println("yaml file: ", yamlFile) conf.LoadConfig(yamlFile) @@ -63,7 +63,7 @@ func main() { if wn.MsgCounter.InsertCounter/CountInsertMsgBaseline != BaselineCounter { wn.WriteWriterLog() - BaselineCounter = wn.MsgCounter.InsertCounter/CountInsertMsgBaseline + BaselineCounter = wn.MsgCounter.InsertCounter / CountInsertMsgBaseline } if msgLength > 0 { diff --git a/internal/allocator/allocator.go b/internal/allocator/allocator.go new file mode 100644 index 0000000000..76e1a0f2a5 --- /dev/null +++ b/internal/allocator/allocator.go @@ -0,0 +1,207 @@ +package allocator + +import ( + "context" + "errors" + "fmt" + "log" + "sync" + "time" + + "github.com/zilliztech/milvus-distributed/internal/proto/masterpb" + "google.golang.org/grpc" +) + +const ( + maxMergeRequests = 10000 +) + +type request interface { + Wait() + Notify(error) + IsValid() bool +} + +type baseRequest struct { + done chan error + valid bool +} + +func (req *baseRequest) Wait() { + err := <-req.done + req.valid = err == nil +} + +func (req *baseRequest) IsValid() bool { + return req.valid +} + +func (req *baseRequest) Notify(err error) { + req.done <- err +} + +type idRequest struct { + baseRequest + id uint64 + count uint32 +} + +func (req *idRequest) Wait() { + req.baseRequest.Wait() +} + +type tsoRequest struct { + baseRequest + timestamp uint64 + count uint32 +} + +func (req *tsoRequest) Wait() { + req.baseRequest.Wait() +} + +type tickerChan interface { + Chan() <-chan time.Time + Close() + Init() + Reset() +} + +type emptyTicker struct { + tChan <-chan time.Time +} + +func (t *emptyTicker) Chan() <-chan time.Time { + return t.tChan +} + +func (t *emptyTicker) Init() { +} + +func (t *emptyTicker) Reset() { +} + +func (t *emptyTicker) Close() { +} + +type ticker struct { + ticker *time.Ticker + updateInterval time.Duration // +} + +func (t *ticker) Init() { + t.ticker = time.NewTicker(t.updateInterval) +} + +func (t *ticker) Reset() { + t.ticker.Reset(t.updateInterval) +} + +func (t *ticker) Close() { + t.ticker.Stop() +} + +func (t *ticker) Chan() <-chan time.Time { + return t.ticker.C +} + +type Allocator struct { + reqs chan request + + wg sync.WaitGroup + ctx context.Context + cancel context.CancelFunc + + masterAddress string + masterConn *grpc.ClientConn + masterClient masterpb.MasterClient + countPerRpc uint32 + + tChan tickerChan + syncFunc func() + processFunc func(req request) +} + +func (ta *Allocator) Start() error { + err := ta.connectMaster() + if err != nil { + panic("connect to master failed") + } + ta.tChan.Init() + ta.wg.Add(1) + go ta.mainLoop() + return nil +} + +func (ta *Allocator) connectMaster() error { + log.Printf("Connected to master, master_addr=%s", ta.masterAddress) + ctx, _ := context.WithTimeout(context.Background(), 2*time.Second) + conn, err := grpc.DialContext(ctx, ta.masterAddress, grpc.WithInsecure(), grpc.WithBlock()) + if err != nil { + log.Printf("Connect to master failed, error= %v", err) + return err + } + log.Printf("Connected to master, master_addr=%s", ta.masterAddress) + ta.masterConn = conn + ta.masterClient = masterpb.NewMasterClient(conn) + return nil +} + +func (ta *Allocator) mainLoop() { + defer ta.wg.Done() + + loopCtx, loopCancel := context.WithCancel(ta.ctx) + defer loopCancel() + + defaultSize := maxMergeRequests + 1 + reqs := make([]request, defaultSize) + for { + select { + case <-ta.tChan.Chan(): + ta.sync() + case first := <-ta.reqs: + pendingPlus1 := len(ta.reqs) + 1 + reqs[0] = first + for i := 1; i < pendingPlus1; i++ { + reqs[i] = <-ta.reqs + } + ta.finishRequest(reqs[:pendingPlus1]) + + case <-loopCtx.Done(): + return + } + + } +} + +func (ta *Allocator) sync() { + if ta.syncFunc != nil { + ta.syncFunc() + ta.tChan.Reset() + fmt.Println("synced") + } +} + +func (ta *Allocator) finishRequest(reqs []request) { + for i := 0; i < len(reqs); i++ { + ta.processFunc(reqs[i]) + if reqs[i] != nil { + reqs[i].Notify(nil) + } + } +} + +func (ta *Allocator) revokeRequest(err error) { + n := len(ta.reqs) + for i := 0; i < n; i++ { + req := <-ta.reqs + req.Notify(err) + } +} + +func (ta *Allocator) Close() { + ta.cancel() + ta.wg.Wait() + ta.tChan.Close() + ta.revokeRequest(errors.New("closing")) +} diff --git a/internal/allocator/id_allocator.go b/internal/allocator/id_allocator.go index c5cccaf85d..3b09d183aa 100644 --- a/internal/allocator/id_allocator.go +++ b/internal/allocator/id_allocator.go @@ -1,35 +1,79 @@ package allocator import ( - "github.com/zilliztech/milvus-distributed/internal/util/typeutil" + "context" + "fmt" + "log" + "time" + + "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" ) type IdAllocator struct { + Allocator + + idStart int64 + idEnd int64 +} + +func NewIdAllocator(ctx context.Context) (*IdAllocator, error) { + ctx1, cancel := context.WithCancel(ctx) + a := &IdAllocator{ + Allocator: Allocator{reqs: make(chan request, maxMergeRequests), + ctx: ctx1, + cancel: cancel, + }, + } + a.tChan = &emptyTicker{} + a.Allocator.syncFunc = a.syncId + a.Allocator.processFunc = a.processFunc + return a, nil +} + +func (ta *IdAllocator) syncId() { + fmt.Println("syncId") + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + req := &internalpb.IdRequest{ + PeerId: 1, + Role: internalpb.PeerRole_Proxy, + Count: ta.countPerRpc, + } + resp, err := ta.masterClient.AllocId(ctx, req) + + cancel() + if err != nil { + log.Panic("syncId Failed!!!!!") + return + } + ta.idStart = resp.GetId() + ta.idEnd = ta.idStart + int64(resp.GetCount()) } -func (allocator *IdAllocator) Initialize() error { - return nil +func (ta *IdAllocator) processFunc(req request) { + idRequest := req.(*idRequest) + idRequest.id = 1 + fmt.Println("process Id") } -func (allocator *IdAllocator) Start() error{ - return nil -} -func (allocator *IdAllocator) Close() error{ - return nil +func (ta *IdAllocator) AllocOne() (int64, error) { + ret, _, err := ta.Alloc(1) + if err != nil { + return 0, err + } + return ret, nil } -func (allocator *IdAllocator) AllocOne() typeutil.Id { - return 1 +func (ta *IdAllocator) Alloc(count uint32) (int64, int64, error) { + req := &idRequest{baseRequest: baseRequest{done: make(chan error), valid: false}} + + req.count = count + ta.reqs <- req + req.Wait() + + if !req.IsValid() { + return 0, 0, nil + } + start, count := int64(req.id), req.count + return start, start + int64(count), nil } - -func (allocator *IdAllocator) Alloc(count uint32) ([]typeutil.Id, error){ - return make([]typeutil.Id, count), nil -} - - -func NewIdAllocator() *IdAllocator{ - return &IdAllocator{} -} - - diff --git a/internal/allocator/timestamp_allocator.go b/internal/allocator/timestamp_allocator.go index 18f1b4e42e..f0d2d4359a 100644 --- a/internal/allocator/timestamp_allocator.go +++ b/internal/allocator/timestamp_allocator.go @@ -1,32 +1,98 @@ package allocator import ( - "github.com/zilliztech/milvus-distributed/internal/util/typeutil" + "context" + "fmt" + "log" + "time" + + "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" ) -type TimestampAllocator struct {} +type Timestamp = uint64 -func (allocator *TimestampAllocator) Start() error{ - return nil +const ( + tsCountPerRPC = 2 << 18 * 10 + defaultUpdateInterval = 1000 * time.Millisecond +) + +type TimestampAllocator struct { + Allocator + lastTsBegin uint64 + lastTsEnd uint64 } -func (allocator *TimestampAllocator) Close() error{ - return nil +func NewTimestampAllocator(ctx context.Context) (*TimestampAllocator, error) { + ctx1, cancel := context.WithCancel(ctx) + a := &TimestampAllocator{ + Allocator: Allocator{reqs: make(chan request, maxMergeRequests), + ctx: ctx1, + cancel: cancel, + }, + } + a.tChan = &ticker{ + updateInterval: time.Second, + } + a.Allocator.syncFunc = a.syncTs + a.Allocator.processFunc = a.processFunc + return a, nil } -func (allocator *TimestampAllocator) AllocOne() (typeutil.Timestamp, error){ - ret, err := allocator.Alloc(1) - if err != nil{ - return typeutil.ZeroTimestamp, err +func (ta *TimestampAllocator) syncTs() { + fmt.Println("sync TS") + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + req := &internalpb.TsoRequest{ + PeerId: 1, + Role: internalpb.PeerRole_Proxy, + Count: ta.countPerRpc, + } + resp, err := ta.masterClient.AllocTimestamp(ctx, req) + + cancel() + if err != nil { + log.Panic("syncId Failed!!!!!") + return + } + ta.lastTsBegin = resp.GetTimestamp() + ta.lastTsEnd = ta.lastTsBegin + uint64(resp.GetCount()) +} + +func (ta *TimestampAllocator) processFunc(req request) { + if req == nil { + fmt.Println("Occur nil!!!!") + return + } + tsoRequest := req.(*tsoRequest) + tsoRequest.timestamp = 1 + fmt.Println("process tso") +} + +func (ta *TimestampAllocator) AllocOne() (Timestamp, error) { + ret, err := ta.Alloc(1) + if err != nil { + return 0, err } return ret[0], nil } -func (allocator *TimestampAllocator) Alloc(count uint32) ([]typeutil.Timestamp, error){ - // to do lock and accuire more by grpc request - return make([]typeutil.Timestamp, count), nil -} +func (ta *TimestampAllocator) Alloc(count uint32) ([]Timestamp, error) { + //req := tsoReqPool.Get().(*tsoRequest) + req := &tsoRequest{ + baseRequest: baseRequest{done: make(chan error), valid: false}, + } + req.count = count + ta.reqs <- req + fmt.Println("YYYYY ", len(ta.reqs)) + req.Wait() -func NewTimestampAllocator() *TimestampAllocator{ - return &TimestampAllocator{} + if !req.IsValid() { + return nil, nil + } + + start, count := req.timestamp, req.count + var ret []Timestamp + for i := uint32(0); i < count; i++ { + ret = append(ret, start+uint64(i)) + } + return ret, nil } diff --git a/internal/conf/conf_test.go b/internal/conf/conf_test.go index e133ee5e5e..9255a701b8 100644 --- a/internal/conf/conf_test.go +++ b/internal/conf/conf_test.go @@ -8,4 +8,3 @@ import ( func TestMain(m *testing.M) { fmt.Printf("Result: %v\n", Config) } - diff --git a/internal/errors/errors.go b/internal/errors/errors.go index 7950dd17c6..f6d6937ad6 100644 --- a/internal/errors/errors.go +++ b/internal/errors/errors.go @@ -1,4 +1,3 @@ - package errors import ( diff --git a/internal/kv/etcd_kv.go b/internal/kv/etcd_kv.go index 2b8ba20d3d..2e6bd75a11 100644 --- a/internal/kv/etcd_kv.go +++ b/internal/kv/etcd_kv.go @@ -2,11 +2,12 @@ package kv import ( "context" - "github.com/zilliztech/milvus-distributed/internal/errors" - "go.etcd.io/etcd/clientv3" "log" "path" "time" + + "github.com/zilliztech/milvus-distributed/internal/errors" + "go.etcd.io/etcd/clientv3" ) const ( diff --git a/internal/kv/etcd_kv_test.go b/internal/kv/etcd_kv_test.go index 1edbcc7f5e..fabe9f7629 100644 --- a/internal/kv/etcd_kv_test.go +++ b/internal/kv/etcd_kv_test.go @@ -1,10 +1,11 @@ package kv import ( - "github.com/stretchr/testify/assert" - "go.etcd.io/etcd/clientv3" "path" "testing" + + "github.com/stretchr/testify/assert" + "go.etcd.io/etcd/clientv3" ) func TestEtcdKV_Load(t *testing.T) { diff --git a/internal/master/collection/collection.go b/internal/master/collection/collection.go index a383604250..7e80f201db 100644 --- a/internal/master/collection/collection.go +++ b/internal/master/collection/collection.go @@ -3,29 +3,29 @@ package collection import ( "time" - "github.com/zilliztech/milvus-distributed/internal/proto/etcdpb" - "github.com/zilliztech/milvus-distributed/internal/proto/schemapb" "github.com/gogo/protobuf/proto" jsoniter "github.com/json-iterator/go" + "github.com/zilliztech/milvus-distributed/internal/proto/etcdpb" + "github.com/zilliztech/milvus-distributed/internal/proto/schemapb" ) var json = jsoniter.ConfigCompatibleWithStandardLibrary type Collection struct { - ID int64 `json:"id"` + ID int64 `json:"id"` Name string `json:"name"` CreateTime uint64 `json:"creat_time"` Schema []FieldMeta `json:"schema"` // ExtraSchema []FieldMeta `json:"extra_schema"` - SegmentIDs []int64 `json:"segment_ids"` - PartitionTags []string `json:"partition_tags"` - GrpcMarshalString string `json:"grpc_marshal_string"` + SegmentIDs []int64 `json:"segment_ids"` + PartitionTags []string `json:"partition_tags"` + GrpcMarshalString string `json:"grpc_marshal_string"` } type FieldMeta struct { - FieldName string `json:"field_name"` + FieldName string `json:"field_name"` Type schemapb.DataType `json:"type"` - DIM int64 `json:"dimension"` + DIM int64 `json:"dimension"` } func GrpcMarshal(c *Collection) *Collection { @@ -38,8 +38,8 @@ func GrpcMarshal(c *Collection) *Collection { schemaSlice := []*schemapb.FieldSchema{} for _, v := range c.Schema { newpbMeta := &schemapb.FieldSchema{ - Name: v.FieldName, - DataType: schemapb.DataType(v.Type), //czs_tag + Name: v.FieldName, + DataType: schemapb.DataType(v.Type), //czs_tag } schemaSlice = append(schemaSlice, newpbMeta) } diff --git a/internal/master/collection/collection_test.go b/internal/master/collection/collection_test.go index 70abc6a384..014559a5b1 100644 --- a/internal/master/collection/collection_test.go +++ b/internal/master/collection/collection_test.go @@ -3,8 +3,9 @@ package collection import ( "testing" "time" - "github.com/zilliztech/milvus-distributed/internal/proto/schemapb" + "github.com/stretchr/testify/assert" + "github.com/zilliztech/milvus-distributed/internal/proto/schemapb" ) var ( diff --git a/internal/master/collection_task.go b/internal/master/collection_task.go index c26d6a16e8..d98ae2482d 100644 --- a/internal/master/collection_task.go +++ b/internal/master/collection_task.go @@ -3,13 +3,14 @@ package master import ( "encoding/json" "errors" + "log" + "strconv" + "github.com/zilliztech/milvus-distributed/internal/proto/commonpb" "github.com/zilliztech/milvus-distributed/internal/proto/etcdpb" "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" "github.com/zilliztech/milvus-distributed/internal/proto/schemapb" "github.com/zilliztech/milvus-distributed/internal/proto/servicepb" - "log" - "strconv" ) const collectionMetaPrefix = "collection/" diff --git a/internal/master/controller/collection.go b/internal/master/controller/collection.go index 6c2ed08900..f860efa7c9 100644 --- a/internal/master/controller/collection.go +++ b/internal/master/controller/collection.go @@ -5,21 +5,20 @@ import ( "strconv" "time" - "github.com/zilliztech/milvus-distributed/internal/conf" - "github.com/zilliztech/milvus-distributed/internal/master/collection" - "github.com/zilliztech/milvus-distributed/internal/proto/schemapb" - "github.com/zilliztech/milvus-distributed/internal/allocator" - "github.com/zilliztech/milvus-distributed/internal/kv" - "github.com/zilliztech/milvus-distributed/internal/master/segment" -) + "github.com/zilliztech/milvus-distributed/internal/master/id" -var IdAllocator *allocator.IdAllocator = allocator.NewIdAllocator() + "github.com/zilliztech/milvus-distributed/internal/conf" + "github.com/zilliztech/milvus-distributed/internal/kv" + "github.com/zilliztech/milvus-distributed/internal/master/collection" + "github.com/zilliztech/milvus-distributed/internal/master/segment" + "github.com/zilliztech/milvus-distributed/internal/proto/schemapb" +) func CollectionController(ch chan *schemapb.CollectionSchema, kvbase kv.Base, errch chan error) { for collectionMeta := range ch { - sID := IdAllocator.AllocOne() - cID := IdAllocator.AllocOne() - s2ID := IdAllocator.AllocOne() + sID, _ := id.AllocOne() + cID, _ := id.AllocOne() + s2ID, _ := id.AllocOne() fieldMetas := []*schemapb.FieldSchema{} if collectionMeta.Fields != nil { fieldMetas = collectionMeta.Fields @@ -55,8 +54,8 @@ func CollectionController(ch chan *schemapb.CollectionSchema, kvbase kv.Base, er } func WriteCollection2Datastore(collectionMeta *schemapb.CollectionSchema, kvbase kv.Base) error { - sID := IdAllocator.AllocOne() - cID := IdAllocator.AllocOne() + sID, _ := id.AllocOne() + cID, _ := id.AllocOne() fieldMetas := []*schemapb.FieldSchema{} if collectionMeta.Fields != nil { fieldMetas = collectionMeta.Fields diff --git a/internal/master/controller/segment.go b/internal/master/controller/segment.go index 19fb6e4f98..1ae8713f86 100644 --- a/internal/master/controller/segment.go +++ b/internal/master/controller/segment.go @@ -5,13 +5,14 @@ import ( "strconv" "time" - "github.com/zilliztech/milvus-distributed/internal/conf" - "github.com/zilliztech/milvus-distributed/internal/master/collection" - "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" - "github.com/zilliztech/milvus-distributed/internal/kv" - "github.com/zilliztech/milvus-distributed/internal/master/segment" -) + "github.com/zilliztech/milvus-distributed/internal/master/id" + "github.com/zilliztech/milvus-distributed/internal/conf" + "github.com/zilliztech/milvus-distributed/internal/kv" + "github.com/zilliztech/milvus-distributed/internal/master/collection" + "github.com/zilliztech/milvus-distributed/internal/master/segment" + "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" +) func ComputeCloseTime(ss internalpb.SegmentStatistics, kvbase kv.Base) error { if int(ss.MemorySize) > int(conf.Config.Master.SegmentThreshole*0.8) { @@ -38,7 +39,7 @@ func ComputeCloseTime(ss internalpb.SegmentStatistics, kvbase kv.Base) error { } kvbase.Save("segment/"+strconv.Itoa(int(ss.SegmentId)), updateData) //create new segment - newSegID := IdAllocator.AllocOne() + newSegID, _ := id.AllocOne() newSeg := segment.NewSegment(newSegID, seg.CollectionID, seg.CollectionName, "default", seg.ChannelStart, seg.ChannelEnd, currentTime, time.Unix(1<<36-1, 0)) newSegData, err := segment.Segment2JSON(*&newSeg) if err != nil { @@ -63,4 +64,3 @@ func ComputeCloseTime(ss internalpb.SegmentStatistics, kvbase kv.Base) error { } return nil } - diff --git a/internal/master/controller/segment_test.go b/internal/master/controller/segment_test.go index 2f40c97f99..735c6e4082 100644 --- a/internal/master/controller/segment_test.go +++ b/internal/master/controller/segment_test.go @@ -28,7 +28,7 @@ func TestComputeClosetTime(t *testing.T) { var news internalpb.SegmentStatistics for i := 0; i < 10; i++ { news = internalpb.SegmentStatistics{ - SegmentId: int64(6875940398055133887), + SegmentId: int64(6875940398055133887), MemorySize: int64(i * 1000), } ComputeCloseTime(news, kvbase) diff --git a/internal/master/grpc_service.go b/internal/master/grpc_service.go index 4d6ec4cfe2..6695d1982e 100644 --- a/internal/master/grpc_service.go +++ b/internal/master/grpc_service.go @@ -2,17 +2,13 @@ package master import ( "context" + "time" + "github.com/zilliztech/milvus-distributed/internal/errors" + "github.com/zilliztech/milvus-distributed/internal/master/id" "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/servicepb" - "go.uber.org/zap" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" - "io" - "log" - "time" ) const slowThreshold = 5 * time.Millisecond @@ -340,38 +336,43 @@ func (s *Master) ShowPartitions(ctx context.Context, in *internalpb.ShowPartitio return t.(*showPartitionTask).stringListResponse, nil } - //----------------------------------------Internal GRPC Service-------------------------------- -// Tso implements gRPC PDServer. -func (s *Master) Tso(stream masterpb.Master_TsoServer) error { - for { - request, err := stream.Recv() - if err == io.EOF { - return nil - } - if err != nil { - return errors.WithStack(err) - } - start := time.Now() +func (s *Master) AllocTimestamp(ctx context.Context, request *internalpb.TsoRequest) (*internalpb.TsoResponse, error) { + count := request.GetCount() + ts, err := s.tsoAllocator.GenerateTSO(count) - count := request.GetCount() - ts, err := s.tsoAllocator.GenerateTSO(count) - if err != nil { - return status.Errorf(codes.Unknown, err.Error()) - } - - elapsed := time.Since(start) - if elapsed > slowThreshold { - log.Println("get timestamp too slow", zap.Duration("cost", elapsed)) - } - response := &internalpb.TsoResponse{ - Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}, - Timestamp: ts, - Count: count, - } - if err := stream.Send(response); err != nil { - return errors.WithStack(err) - } + if err != nil { + return &internalpb.TsoResponse{ + Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR}, + }, err } -} \ No newline at end of file + + response := &internalpb.TsoResponse{ + Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR}, + Timestamp: ts, + Count: count, + } + + return response, nil +} + +func (s *Master) AllocId(ctx context.Context, request *internalpb.IdRequest) (*internalpb.IdResponse, error) { + panic("implement me") + count := request.GetCount() + ts, err := id.AllocOne() + + if err != nil { + return &internalpb.IdResponse{ + Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR}, + }, err + } + + response := &internalpb.IdResponse{ + Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR}, + Id: ts, + Count: count, + } + + return response, nil +} diff --git a/internal/master/id/id.go b/internal/master/id/id.go index 609dc7a716..5e7576ba7a 100644 --- a/internal/master/id/id.go +++ b/internal/master/id/id.go @@ -17,16 +17,15 @@ import ( "github.com/zilliztech/milvus-distributed/internal/master/tso" ) - // GlobalTSOAllocator is the global single point TSO allocator. type GlobalIdAllocator struct { allocator tso.Allocator } -func NewGlobalIdAllocator() *GlobalIdAllocator { - return &GlobalIdAllocator{ - allocator: tso.NewGlobalTSOAllocator("idTimestamp"), - } +//func getID + +var allocator GlobalIdAllocator = GlobalIdAllocator{ + allocator: tso.NewGlobalTSOAllocator("idTimestamp"), } // Initialize will initialize the created global TSO allocator. @@ -36,9 +35,9 @@ func (gia *GlobalIdAllocator) Initialize() error { // GenerateTSO is used to generate a given number of TSOs. // Make sure you have initialized the TSO allocator before calling. -func (gia *GlobalIdAllocator) Generate(count uint32) (int64, int64, error) { - timestamp, err:= gia.allocator.GenerateTSO(count) - if err != nil{ +func (gia *GlobalIdAllocator) Alloc(count uint32) (int64, int64, error) { + timestamp, err := gia.allocator.GenerateTSO(count) + if err != nil { return 0, 0, err } idStart := int64(timestamp) @@ -46,3 +45,19 @@ func (gia *GlobalIdAllocator) Generate(count uint32) (int64, int64, error) { return idStart, idEnd, nil } +func (gia *GlobalIdAllocator) AllocOne() (int64, error) { + timestamp, err := gia.allocator.GenerateTSO(1) + if err != nil { + return 0, err + } + idStart := int64(timestamp) + return idStart, nil +} + +func AllocOne() (int64, error) { + return allocator.AllocOne() +} + +func Alloc(count uint32) (int64, int64, error) { + return allocator.Alloc(count) +} diff --git a/internal/master/master.go b/internal/master/master.go index e79aef6cf0..dea7061fb3 100644 --- a/internal/master/master.go +++ b/internal/master/master.go @@ -3,6 +3,14 @@ package master import ( "context" "fmt" + "log" + "math/rand" + "net" + "strconv" + "sync" + "sync/atomic" + "time" + "github.com/apache/pulsar-client-go/pulsar" "github.com/golang/protobuf/proto" "github.com/zilliztech/milvus-distributed/internal/conf" @@ -12,13 +20,6 @@ import ( "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" "github.com/zilliztech/milvus-distributed/internal/proto/masterpb" "google.golang.org/grpc" - "log" - "math/rand" - "net" - "strconv" - "sync" - "sync/atomic" - "time" "github.com/zilliztech/milvus-distributed/internal/master/tso" "go.etcd.io/etcd/clientv3" diff --git a/internal/master/meta_table.go b/internal/master/meta_table.go index 8ca110bd8c..e4245cfbd0 100644 --- a/internal/master/meta_table.go +++ b/internal/master/meta_table.go @@ -1,12 +1,13 @@ package master import ( + "strconv" + "sync" + "github.com/golang/protobuf/proto" "github.com/zilliztech/milvus-distributed/internal/errors" "github.com/zilliztech/milvus-distributed/internal/kv" pb "github.com/zilliztech/milvus-distributed/internal/proto/etcdpb" - "strconv" - "sync" ) type metaTable struct { diff --git a/internal/master/meta_table_test.go b/internal/master/meta_table_test.go index a432a8d44e..af66fa0146 100644 --- a/internal/master/meta_table_test.go +++ b/internal/master/meta_table_test.go @@ -1,12 +1,13 @@ package master import ( + "testing" + "github.com/stretchr/testify/assert" "github.com/zilliztech/milvus-distributed/internal/kv" pb "github.com/zilliztech/milvus-distributed/internal/proto/etcdpb" "github.com/zilliztech/milvus-distributed/internal/proto/schemapb" "go.etcd.io/etcd/clientv3" - "testing" ) func TestMetaTable_DeletePartition(t *testing.T) { diff --git a/internal/master/partition_task.go b/internal/master/partition_task.go index 933726494f..1e8b9c87e4 100644 --- a/internal/master/partition_task.go +++ b/internal/master/partition_task.go @@ -3,11 +3,12 @@ package master import ( "encoding/json" "errors" + "log" + "strconv" + "github.com/zilliztech/milvus-distributed/internal/proto/commonpb" "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" "github.com/zilliztech/milvus-distributed/internal/proto/servicepb" - "log" - "strconv" ) const partitionMetaPrefix = "partition/" @@ -195,7 +196,7 @@ func (t *describePartitionTask) Execute() error { partitionName := t.req.PartitionName - description := servicepb.PartitionDescription { + description := servicepb.PartitionDescription{ Status: &commonpb.Status{ ErrorCode: commonpb.ErrorCode_SUCCESS, }, @@ -237,7 +238,7 @@ func (t *showPartitionTask) Execute() error { } } - stringListResponse := servicepb.StringListResponse { + stringListResponse := servicepb.StringListResponse{ Status: &commonpb.Status{ ErrorCode: commonpb.ErrorCode_SUCCESS, }, diff --git a/internal/master/segment/segment.go b/internal/master/segment/segment.go index 7aaee83367..ced52d640e 100644 --- a/internal/master/segment/segment.go +++ b/internal/master/segment/segment.go @@ -9,15 +9,15 @@ import ( var json = jsoniter.ConfigCompatibleWithStandardLibrary type Segment struct { - SegmentID int64 `json:"segment_id"` - CollectionID int64 `json:"collection_id"` - PartitionTag string `json:"partition_tag"` - ChannelStart int `json:"channel_start"` - ChannelEnd int `json:"channel_end"` - OpenTimeStamp uint64 `json:"open_timestamp"` - CloseTimeStamp uint64 `json:"close_timestamp"` - CollectionName string `json:"collection_name"` - Rows int64 `json:"rows"` + SegmentID int64 `json:"segment_id"` + CollectionID int64 `json:"collection_id"` + PartitionTag string `json:"partition_tag"` + ChannelStart int `json:"channel_start"` + ChannelEnd int `json:"channel_end"` + OpenTimeStamp uint64 `json:"open_timestamp"` + CloseTimeStamp uint64 `json:"close_timestamp"` + CollectionName string `json:"collection_name"` + Rows int64 `json:"rows"` } func NewSegment(id int64, collectioID int64, cName string, ptag string, chStart int, chEnd int, openTime time.Time, closeTime time.Time) Segment { diff --git a/internal/master/task.go b/internal/master/task.go index a314e69833..dbf0de0692 100644 --- a/internal/master/task.go +++ b/internal/master/task.go @@ -2,6 +2,7 @@ package master import ( "context" + "github.com/zilliztech/milvus-distributed/internal/errors" "github.com/zilliztech/milvus-distributed/internal/kv" "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" diff --git a/internal/master/timesync/timesync.go b/internal/master/timesync/timesync.go index 338b2d13f2..a8c7939489 100644 --- a/internal/master/timesync/timesync.go +++ b/internal/master/timesync/timesync.go @@ -2,13 +2,14 @@ package timesync import ( "context" - "github.com/zilliztech/milvus-distributed/internal/conf" "log" "sort" "strconv" "sync" "time" + "github.com/zilliztech/milvus-distributed/internal/conf" + "github.com/apache/pulsar-client-go/pulsar" "github.com/golang/protobuf/proto" pb "github.com/zilliztech/milvus-distributed/internal/proto/message" @@ -29,7 +30,7 @@ type TimeTickReader struct { ctx context.Context } -func (r *TimeTickReader) Start(){ +func (r *TimeTickReader) Start() { go r.readTimeTick() go r.timeSync() } @@ -113,7 +114,6 @@ func (r *TimeTickReader) sendEOFMsg(ctx context.Context, msg *pulsar.ProducerMes wg.Done() } - func TimeTickService() { timeTickTopic := "timeTick" timeTickSubName := "master" diff --git a/internal/master/tso/global_allocator.go b/internal/master/tso/global_allocator.go index 90d7b9b716..0ab4fa77b7 100644 --- a/internal/master/tso/global_allocator.go +++ b/internal/master/tso/global_allocator.go @@ -14,13 +14,14 @@ package tso import ( - "github.com/zilliztech/milvus-distributed/internal/conf" - "github.com/zilliztech/milvus-distributed/internal/util/tsoutil" - "go.etcd.io/etcd/clientv3" "strconv" "sync/atomic" "time" + "github.com/zilliztech/milvus-distributed/internal/conf" + "github.com/zilliztech/milvus-distributed/internal/util/tsoutil" + "go.etcd.io/etcd/clientv3" + "github.com/pingcap/log" "github.com/zilliztech/milvus-distributed/internal/errors" "github.com/zilliztech/milvus-distributed/internal/util/typeutil" @@ -62,14 +63,14 @@ func NewGlobalTSOAllocator(key string) Allocator { DialTimeout: 5 * time.Second, }) - var saveInterval time.Duration = 3 *time.Second + var saveInterval time.Duration = 3 * time.Second return &GlobalTSOAllocator{ timestampOracle: ×tampOracle{ client: client, rootPath: conf.Config.Etcd.Rootpath, saveInterval: saveInterval, - maxResetTSGap: func() time.Duration { return 3 *time.Second}, - key: key, + maxResetTSGap: func() time.Duration { return 3 * time.Second }, + key: key, }, } } diff --git a/internal/master/tso/tso.go b/internal/master/tso/tso.go index 88591962c8..8feb7efa8d 100644 --- a/internal/master/tso/tso.go +++ b/internal/master/tso/tso.go @@ -14,13 +14,14 @@ package tso import ( - "go.uber.org/zap" "log" "path" "sync/atomic" "time" "unsafe" + "go.uber.org/zap" + "github.com/zilliztech/milvus-distributed/internal/errors" "github.com/zilliztech/milvus-distributed/internal/util/etcdutil" "github.com/zilliztech/milvus-distributed/internal/util/tsoutil" @@ -49,7 +50,7 @@ type atomicObject struct { type timestampOracle struct { client *clientv3.Client rootPath string - key string + key string // TODO: remove saveInterval saveInterval time.Duration maxResetTSGap func() time.Duration @@ -75,11 +76,11 @@ func (t *timestampOracle) loadTimestamp() (time.Time, error) { // save timestamp, if lastTs is 0, we think the timestamp doesn't exist, so create it, // otherwise, update it. -func (t *timestampOracle) saveTimestamp( ts time.Time) error { +func (t *timestampOracle) saveTimestamp(ts time.Time) error { key := t.getTimestampPath() data := typeutil.Uint64ToBytes(uint64(ts.UnixNano())) err := errors.New("") - println("%v,%v",key, data) + println("%v,%v", key, data) //resp, err := leadership.LeaderTxn(). // Then(clientv3.OpPut(key, string(data))). // Commit() @@ -127,7 +128,7 @@ func (t *timestampOracle) SyncTimestamp() error { } // ResetUserTimestamp update the physical part with specified tso. -func (t *timestampOracle) ResetUserTimestamp( tso uint64) error { +func (t *timestampOracle) ResetUserTimestamp(tso uint64) error { //if !leadership.Check() { // return errors.New("Setup timestamp failed, lease expired") //} @@ -145,7 +146,7 @@ func (t *timestampOracle) ResetUserTimestamp( tso uint64) error { } save := next.Add(t.saveInterval) - if err := t.saveTimestamp( save); err != nil { + if err := t.saveTimestamp(save); err != nil { return err } update := &atomicObject{ @@ -194,7 +195,7 @@ func (t *timestampOracle) UpdateTimestamp() error { // The time window needs to be updated and saved to etcd. if typeutil.SubTimeByWallClock(t.lastSavedTime.Load().(time.Time), next) <= updateTimestampGuard { save := next.Add(t.saveInterval) - if err := t.saveTimestamp( save); err != nil { + if err := t.saveTimestamp(save); err != nil { return err } } diff --git a/internal/msgclient/msgclient.go b/internal/msgclient/msgclient.go index 006db608d8..726d7ad5dc 100644 --- a/internal/msgclient/msgclient.go +++ b/internal/msgclient/msgclient.go @@ -7,11 +7,11 @@ import ( "strconv" "github.com/apache/pulsar-client-go/pulsar" + "github.com/golang/protobuf/proto" "github.com/zilliztech/milvus-distributed/internal/conf" "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message" "github.com/zilliztech/milvus-distributed/internal/timesync" - "github.com/golang/protobuf/proto" ) type MessageType int diff --git a/internal/msgstream/marshaler.go b/internal/msgstream/marshaler.go index 632f39a78c..cb6f7d36c2 100644 --- a/internal/msgstream/marshaler.go +++ b/internal/msgstream/marshaler.go @@ -59,7 +59,7 @@ func (im *InsertMarshaler) Marshal(input *TsMsg) ([]byte, commonPb.Status) { func (im *InsertMarshaler) Unmarshal(input []byte) (*TsMsg, commonPb.Status) { insertRequest := internalPb.InsertRequest{} err := proto.Unmarshal(input, &insertRequest) - insertTask := InsertTask{InsertRequest:insertRequest} + insertTask := InsertTask{InsertRequest: insertRequest} if err != nil { return nil, commonPb.Status{ErrorCode: commonPb.ErrorCode_UNEXPECTED_ERROR} @@ -85,7 +85,7 @@ func (dm *DeleteMarshaler) Marshal(input *TsMsg) ([]byte, commonPb.Status) { func (dm *DeleteMarshaler) Unmarshal(input []byte) (*TsMsg, commonPb.Status) { deleteRequest := internalPb.DeleteRequest{} err := proto.Unmarshal(input, &deleteRequest) - deleteTask := DeleteTask{DeleteRequest:deleteRequest} + deleteTask := DeleteTask{DeleteRequest: deleteRequest} if err != nil { return nil, commonPb.Status{ErrorCode: commonPb.ErrorCode_UNEXPECTED_ERROR} } @@ -160,7 +160,7 @@ func (tm *TimeSyncMarshaler) Marshal(input *TsMsg) ([]byte, commonPb.Status) { func (tm *TimeSyncMarshaler) Unmarshal(input []byte) (*TsMsg, commonPb.Status) { timeSyncMsg := internalPb.TimeSyncMsg{} err := proto.Unmarshal(input, &timeSyncMsg) - timeSyncTask := TimeSyncTask{TimeSyncMsg:timeSyncMsg} + timeSyncTask := TimeSyncTask{TimeSyncMsg: timeSyncMsg} if err != nil { return nil, commonPb.Status{ErrorCode: commonPb.ErrorCode_UNEXPECTED_ERROR} } diff --git a/internal/msgstream/msgstream.go b/internal/msgstream/msgstream.go index fec2de71bd..17f3ccdb3e 100644 --- a/internal/msgstream/msgstream.go +++ b/internal/msgstream/msgstream.go @@ -2,17 +2,17 @@ package msgstream import ( "context" - "github.com/apache/pulsar-client-go/pulsar" - commonPb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb" "log" "sync" + + "github.com/apache/pulsar-client-go/pulsar" + commonPb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb" + "github.com/zilliztech/milvus-distributed/internal/util/typeutil" ) -type Timestamp uint64 - type MsgPack struct { - BeginTs Timestamp - EndTs Timestamp + BeginTs typeutil.Timestamp + EndTs typeutil.Timestamp Msgs []*TsMsg } @@ -29,12 +29,12 @@ type MsgStream interface { } type PulsarMsgStream struct { - client *pulsar.Client - producers []*pulsar.Producer - consumers []*pulsar.Consumer - repackFunc RepackFunc // return a map from produceChannel idx to *MsgPack + client *pulsar.Client + producers []*pulsar.Producer + consumers []*pulsar.Consumer + repackFunc RepackFunc // return a map from produceChannel idx to *MsgPack - receiveBuf chan *MsgPack + receiveBuf chan *MsgPack msgMarshaler *TsMsgMarshaler msgUnmarshaler *TsMsgMarshaler @@ -86,7 +86,7 @@ func (ms *PulsarMsgStream) SetRepackFunc(repackFunc RepackFunc) { ms.repackFunc = repackFunc } -func (ms *PulsarMsgStream) Start(){ +func (ms *PulsarMsgStream) Start() { go ms.bufMsgPackToChannel() } @@ -110,10 +110,9 @@ func (ms *PulsarMsgStream) InitMsgPackBuf(msgPackBufSize int64) { ms.receiveBuf = make(chan *MsgPack, msgPackBufSize) } - func (ms *PulsarMsgStream) Produce(msgPack *MsgPack) commonPb.Status { tsMsgs := msgPack.Msgs - if len(tsMsgs) <=0 { + if len(tsMsgs) <= 0 { log.Println("receive empty msgPack") return commonPb.Status{ErrorCode: commonPb.ErrorCode_SUCCESS} } @@ -215,17 +214,17 @@ type PulsarTtMsgStream struct { inputBuf []*TsMsg unsolvedBuf []*TsMsg msgPacks []*MsgPack - lastTimeStamp Timestamp + lastTimeStamp typeutil.Timestamp } -func (ms *PulsarTtMsgStream) Start(){ +func (ms *PulsarTtMsgStream) Start() { go ms.bufMsgPackToChannel() } func (ms *PulsarTtMsgStream) bufMsgPackToChannel() { wg := sync.WaitGroup{} wg.Add(len(ms.consumers)) - eofMsgTimeStamp := make(map[int]Timestamp) + eofMsgTimeStamp := make(map[int]typeutil.Timestamp) mu := sync.Mutex{} for i := 0; i < len(ms.consumers); i++ { go ms.findTimeTick(context.Background(), i, eofMsgTimeStamp, &wg, &mu) @@ -259,7 +258,7 @@ func (ms *PulsarTtMsgStream) bufMsgPackToChannel() { func (ms *PulsarTtMsgStream) findTimeTick(ctx context.Context, channelIndex int, - eofMsgMap map[int]Timestamp, + eofMsgMap map[int]typeutil.Timestamp, wg *sync.WaitGroup, mu *sync.Mutex) { for { @@ -289,8 +288,8 @@ func (ms *PulsarTtMsgStream) findTimeTick(ctx context.Context, } } -func checkTimeTickMsg(msg map[int]Timestamp) (Timestamp, bool) { - checkMap := make(map[Timestamp]int) +func checkTimeTickMsg(msg map[int]typeutil.Timestamp) (typeutil.Timestamp, bool) { + checkMap := make(map[typeutil.Timestamp]int) for _, v := range msg { checkMap[v] += 1 } diff --git a/internal/msgstream/msgstream_test.go b/internal/msgstream/msgstream_test.go index b52cc74008..ade3e6aa38 100644 --- a/internal/msgstream/msgstream_test.go +++ b/internal/msgstream/msgstream_test.go @@ -2,9 +2,10 @@ package msgstream import ( "fmt" + "testing" + commonPb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb" internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" - "testing" ) func repackFunc(msgs []*TsMsg, hashKeys [][]int32) map[int32]*MsgPack { @@ -38,7 +39,7 @@ func getTsMsg(msgType MsgType, reqId int64, hashValue int32) *TsMsg { Timestamps: []uint64{1}, } insertMsg := InsertTask{ - HashValues: []int32{hashValue}, + HashValues: []int32{hashValue}, InsertRequest: insertRequest, } tsMsg = insertMsg @@ -49,11 +50,11 @@ func getTsMsg(msgType MsgType, reqId int64, hashValue int32) *TsMsg { CollectionName: "Collection", ChannelId: 1, ProxyId: 1, - Timestamps: []uint64{1}, + Timestamps: []uint64{1}, PrimaryKeys: []int64{1}, } deleteMsg := DeleteTask{ - HashValues: []int32{hashValue}, + HashValues: []int32{hashValue}, DeleteRequest: deleteRequest, } tsMsg = deleteMsg @@ -66,7 +67,7 @@ func getTsMsg(msgType MsgType, reqId int64, hashValue int32) *TsMsg { ResultChannelId: 1, } searchMsg := SearchTask{ - HashValues: []int32{hashValue}, + HashValues: []int32{hashValue}, SearchRequest: searchRequest, } tsMsg = searchMsg @@ -80,7 +81,7 @@ func getTsMsg(msgType MsgType, reqId int64, hashValue int32) *TsMsg { ResultChannelId: 1, } searchResultMsg := SearchResultTask{ - HashValues: []int32{hashValue}, + HashValues: []int32{hashValue}, SearchResult: searchResult, } tsMsg = searchResultMsg @@ -90,7 +91,7 @@ func getTsMsg(msgType MsgType, reqId int64, hashValue int32) *TsMsg { Timestamp: 1, } timeSyncMsg := TimeSyncTask{ - HashValues: []int32{hashValue}, + HashValues: []int32{hashValue}, TimeSyncMsg: timeSyncResult, } tsMsg = timeSyncMsg @@ -106,7 +107,7 @@ func getTsMsg(msgType MsgType, reqId int64, hashValue int32) *TsMsg { Timestamps: []uint64{1}, } insertMsg := InsertTask{ - HashValues: []int32{hashValue}, + HashValues: []int32{hashValue}, InsertRequest: insertRequest, } tsMsg = insertMsg diff --git a/internal/msgstream/newstream_test.go b/internal/msgstream/newstream_test.go index 0069cea889..3437a98aeb 100644 --- a/internal/msgstream/newstream_test.go +++ b/internal/msgstream/newstream_test.go @@ -2,8 +2,9 @@ package msgstream import ( "fmt" - internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" "testing" + + internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" ) func TestNewStream_Insert(t *testing.T) { @@ -216,7 +217,7 @@ func TestNewStream_Insert_TimeTick(t *testing.T) { Timestamps: []uint64{1}, } insertMsg := InsertTask{ - HashValues: []int32{2}, + HashValues: []int32{2}, InsertRequest: insertRequest, } var tsMsg TsMsg = insertMsg @@ -244,8 +245,8 @@ func TestNewStream_Insert_TimeTick(t *testing.T) { fmt.Println("msg type: ", (*v).Type(), ", msg value: ", *v) } } - if receiveCount + 1 >= len(msgPack.Msgs) { + if receiveCount+1 >= len(msgPack.Msgs) { break } } -} \ No newline at end of file +} diff --git a/internal/msgstream/task.go b/internal/msgstream/task.go index 1253671fb2..38be25ca2e 100644 --- a/internal/msgstream/task.go +++ b/internal/msgstream/task.go @@ -2,15 +2,16 @@ package msgstream import ( internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" + . "github.com/zilliztech/milvus-distributed/internal/util/typeutil" ) type MsgType uint32 const ( - kInsert MsgType = 400 - kDelete MsgType = 401 - kSearch MsgType = 500 - kSearchResult MsgType = 1000 + kInsert MsgType = 400 + kDelete MsgType = 401 + kSearch MsgType = 500 + kSearchResult MsgType = 1000 kSegmentStatics MsgType = 1100 kTimeTick MsgType = 1200 @@ -166,11 +167,11 @@ func (srt SearchResultTask) SetTs(ts Timestamp) { srt.Timestamp = uint64(ts) } -func (srt SearchResultTask) BeginTs() Timestamp { +func (srt SearchResultTask) BeginTs() Timestamp { return Timestamp(srt.Timestamp) } -func (srt SearchResultTask) EndTs() Timestamp { +func (srt SearchResultTask) EndTs() Timestamp { return Timestamp(srt.Timestamp) } @@ -196,7 +197,7 @@ func (tst TimeSyncTask) BeginTs() Timestamp { return Timestamp(tst.Timestamp) } -func (tst TimeSyncTask) EndTs() Timestamp { +func (tst TimeSyncTask) EndTs() Timestamp { return Timestamp(tst.Timestamp) } @@ -208,7 +209,6 @@ func (tst TimeSyncTask) HashKeys() []int32 { return tst.HashValues } - ///////////////////////////////////////////Key2Seg////////////////////////////////////////// //type Key2SegTask struct { // internalPb.Key2SegMsg diff --git a/internal/proto/internal_msg.proto b/internal/proto/internal_msg.proto index 77fbd14a33..e22e2e38f8 100644 --- a/internal/proto/internal_msg.proto +++ b/internal/proto/internal_msg.proto @@ -46,6 +46,18 @@ enum PeerRole { } +message IdRequest { + int64 peer_id = 1; + PeerRole role = 2; + uint32 count = 3; +} + +message IdResponse { + common.Status status = 1; + int64 id = 2; + uint32 count = 3; +} + message TsoRequest { int64 peer_id = 1; PeerRole role = 2; diff --git a/internal/proto/internalpb/internal_msg.pb.go b/internal/proto/internalpb/internal_msg.pb.go index 1a4a63aaff..f3e45e2503 100644 --- a/internal/proto/internalpb/internal_msg.pb.go +++ b/internal/proto/internalpb/internal_msg.pb.go @@ -122,6 +122,116 @@ func (PeerRole) EnumDescriptor() ([]byte, []int) { return fileDescriptor_7eb37f6b80b23116, []int{1} } +type IdRequest struct { + 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:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *IdRequest) Reset() { *m = IdRequest{} } +func (m *IdRequest) String() string { return proto.CompactTextString(m) } +func (*IdRequest) ProtoMessage() {} +func (*IdRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_7eb37f6b80b23116, []int{0} +} + +func (m *IdRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_IdRequest.Unmarshal(m, b) +} +func (m *IdRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_IdRequest.Marshal(b, m, deterministic) +} +func (m *IdRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_IdRequest.Merge(m, src) +} +func (m *IdRequest) XXX_Size() int { + return xxx_messageInfo_IdRequest.Size(m) +} +func (m *IdRequest) XXX_DiscardUnknown() { + xxx_messageInfo_IdRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_IdRequest proto.InternalMessageInfo + +func (m *IdRequest) GetPeerId() int64 { + if m != nil { + return m.PeerId + } + return 0 +} + +func (m *IdRequest) GetRole() PeerRole { + if m != nil { + return m.Role + } + return PeerRole_Master +} + +func (m *IdRequest) GetCount() uint32 { + if m != nil { + return m.Count + } + return 0 +} + +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"` + Count uint32 `protobuf:"varint,3,opt,name=count,proto3" json:"count,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *IdResponse) Reset() { *m = IdResponse{} } +func (m *IdResponse) String() string { return proto.CompactTextString(m) } +func (*IdResponse) ProtoMessage() {} +func (*IdResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_7eb37f6b80b23116, []int{1} +} + +func (m *IdResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_IdResponse.Unmarshal(m, b) +} +func (m *IdResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_IdResponse.Marshal(b, m, deterministic) +} +func (m *IdResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_IdResponse.Merge(m, src) +} +func (m *IdResponse) XXX_Size() int { + return xxx_messageInfo_IdResponse.Size(m) +} +func (m *IdResponse) XXX_DiscardUnknown() { + xxx_messageInfo_IdResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_IdResponse proto.InternalMessageInfo + +func (m *IdResponse) GetStatus() *commonpb.Status { + if m != nil { + return m.Status + } + return nil +} + +func (m *IdResponse) GetId() int64 { + if m != nil { + return m.Id + } + return 0 +} + +func (m *IdResponse) GetCount() uint32 { + if m != nil { + return m.Count + } + return 0 +} + type TsoRequest struct { 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"` @@ -135,7 +245,7 @@ func (m *TsoRequest) Reset() { *m = TsoRequest{} } func (m *TsoRequest) String() string { return proto.CompactTextString(m) } func (*TsoRequest) ProtoMessage() {} func (*TsoRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_7eb37f6b80b23116, []int{0} + return fileDescriptor_7eb37f6b80b23116, []int{2} } func (m *TsoRequest) XXX_Unmarshal(b []byte) error { @@ -190,7 +300,7 @@ func (m *TsoResponse) Reset() { *m = TsoResponse{} } func (m *TsoResponse) String() string { return proto.CompactTextString(m) } func (*TsoResponse) ProtoMessage() {} func (*TsoResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_7eb37f6b80b23116, []int{1} + return fileDescriptor_7eb37f6b80b23116, []int{3} } func (m *TsoResponse) XXX_Unmarshal(b []byte) error { @@ -247,7 +357,7 @@ func (m *CreateCollectionRequest) Reset() { *m = CreateCollectionRequest func (m *CreateCollectionRequest) String() string { return proto.CompactTextString(m) } func (*CreateCollectionRequest) ProtoMessage() {} func (*CreateCollectionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_7eb37f6b80b23116, []int{2} + return fileDescriptor_7eb37f6b80b23116, []int{4} } func (m *CreateCollectionRequest) XXX_Unmarshal(b []byte) error { @@ -318,7 +428,7 @@ func (m *DropCollectionRequest) Reset() { *m = DropCollectionRequest{} } func (m *DropCollectionRequest) String() string { return proto.CompactTextString(m) } func (*DropCollectionRequest) ProtoMessage() {} func (*DropCollectionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_7eb37f6b80b23116, []int{3} + return fileDescriptor_7eb37f6b80b23116, []int{5} } func (m *DropCollectionRequest) XXX_Unmarshal(b []byte) error { @@ -389,7 +499,7 @@ func (m *HasCollectionRequest) Reset() { *m = HasCollectionRequest{} } func (m *HasCollectionRequest) String() string { return proto.CompactTextString(m) } func (*HasCollectionRequest) ProtoMessage() {} func (*HasCollectionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_7eb37f6b80b23116, []int{4} + return fileDescriptor_7eb37f6b80b23116, []int{6} } func (m *HasCollectionRequest) XXX_Unmarshal(b []byte) error { @@ -460,7 +570,7 @@ func (m *DescribeCollectionRequest) Reset() { *m = DescribeCollectionReq func (m *DescribeCollectionRequest) String() string { return proto.CompactTextString(m) } func (*DescribeCollectionRequest) ProtoMessage() {} func (*DescribeCollectionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_7eb37f6b80b23116, []int{5} + return fileDescriptor_7eb37f6b80b23116, []int{7} } func (m *DescribeCollectionRequest) XXX_Unmarshal(b []byte) error { @@ -530,7 +640,7 @@ func (m *ShowCollectionRequest) Reset() { *m = ShowCollectionRequest{} } func (m *ShowCollectionRequest) String() string { return proto.CompactTextString(m) } func (*ShowCollectionRequest) ProtoMessage() {} func (*ShowCollectionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_7eb37f6b80b23116, []int{6} + return fileDescriptor_7eb37f6b80b23116, []int{8} } func (m *ShowCollectionRequest) XXX_Unmarshal(b []byte) error { @@ -594,7 +704,7 @@ func (m *CreatePartitionRequest) Reset() { *m = CreatePartitionRequest{} func (m *CreatePartitionRequest) String() string { return proto.CompactTextString(m) } func (*CreatePartitionRequest) ProtoMessage() {} func (*CreatePartitionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_7eb37f6b80b23116, []int{7} + return fileDescriptor_7eb37f6b80b23116, []int{9} } func (m *CreatePartitionRequest) XXX_Unmarshal(b []byte) error { @@ -665,7 +775,7 @@ func (m *DropPartitionRequest) Reset() { *m = DropPartitionRequest{} } func (m *DropPartitionRequest) String() string { return proto.CompactTextString(m) } func (*DropPartitionRequest) ProtoMessage() {} func (*DropPartitionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_7eb37f6b80b23116, []int{8} + return fileDescriptor_7eb37f6b80b23116, []int{10} } func (m *DropPartitionRequest) XXX_Unmarshal(b []byte) error { @@ -736,7 +846,7 @@ func (m *HasPartitionRequest) Reset() { *m = HasPartitionRequest{} } func (m *HasPartitionRequest) String() string { return proto.CompactTextString(m) } func (*HasPartitionRequest) ProtoMessage() {} func (*HasPartitionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_7eb37f6b80b23116, []int{9} + return fileDescriptor_7eb37f6b80b23116, []int{11} } func (m *HasPartitionRequest) XXX_Unmarshal(b []byte) error { @@ -807,7 +917,7 @@ func (m *DescribePartitionRequest) Reset() { *m = DescribePartitionReque func (m *DescribePartitionRequest) String() string { return proto.CompactTextString(m) } func (*DescribePartitionRequest) ProtoMessage() {} func (*DescribePartitionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_7eb37f6b80b23116, []int{10} + return fileDescriptor_7eb37f6b80b23116, []int{12} } func (m *DescribePartitionRequest) XXX_Unmarshal(b []byte) error { @@ -878,7 +988,7 @@ func (m *ShowPartitionRequest) Reset() { *m = ShowPartitionRequest{} } func (m *ShowPartitionRequest) String() string { return proto.CompactTextString(m) } func (*ShowPartitionRequest) ProtoMessage() {} func (*ShowPartitionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_7eb37f6b80b23116, []int{11} + return fileDescriptor_7eb37f6b80b23116, []int{13} } func (m *ShowPartitionRequest) XXX_Unmarshal(b []byte) error { @@ -954,7 +1064,7 @@ func (m *InsertRequest) Reset() { *m = InsertRequest{} } func (m *InsertRequest) String() string { return proto.CompactTextString(m) } func (*InsertRequest) ProtoMessage() {} func (*InsertRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_7eb37f6b80b23116, []int{12} + return fileDescriptor_7eb37f6b80b23116, []int{14} } func (m *InsertRequest) XXX_Unmarshal(b []byte) error { @@ -1062,7 +1172,7 @@ func (m *DeleteRequest) Reset() { *m = DeleteRequest{} } func (m *DeleteRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRequest) ProtoMessage() {} func (*DeleteRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_7eb37f6b80b23116, []int{13} + return fileDescriptor_7eb37f6b80b23116, []int{15} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { @@ -1148,7 +1258,7 @@ func (m *SearchRequest) Reset() { *m = SearchRequest{} } func (m *SearchRequest) String() string { return proto.CompactTextString(m) } func (*SearchRequest) ProtoMessage() {} func (*SearchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_7eb37f6b80b23116, []int{14} + return fileDescriptor_7eb37f6b80b23116, []int{16} } func (m *SearchRequest) XXX_Unmarshal(b []byte) error { @@ -1228,7 +1338,7 @@ func (m *SearchResult) Reset() { *m = SearchResult{} } func (m *SearchResult) String() string { return proto.CompactTextString(m) } func (*SearchResult) ProtoMessage() {} func (*SearchResult) Descriptor() ([]byte, []int) { - return fileDescriptor_7eb37f6b80b23116, []int{15} + return fileDescriptor_7eb37f6b80b23116, []int{17} } func (m *SearchResult) XXX_Unmarshal(b []byte) error { @@ -1310,7 +1420,7 @@ func (m *TimeSyncMsg) Reset() { *m = TimeSyncMsg{} } func (m *TimeSyncMsg) String() string { return proto.CompactTextString(m) } func (*TimeSyncMsg) ProtoMessage() {} func (*TimeSyncMsg) Descriptor() ([]byte, []int) { - return fileDescriptor_7eb37f6b80b23116, []int{16} + return fileDescriptor_7eb37f6b80b23116, []int{18} } func (m *TimeSyncMsg) XXX_Unmarshal(b []byte) error { @@ -1360,7 +1470,7 @@ func (m *Key2Seg) Reset() { *m = Key2Seg{} } func (m *Key2Seg) String() string { return proto.CompactTextString(m) } func (*Key2Seg) ProtoMessage() {} func (*Key2Seg) Descriptor() ([]byte, []int) { - return fileDescriptor_7eb37f6b80b23116, []int{17} + return fileDescriptor_7eb37f6b80b23116, []int{19} } func (m *Key2Seg) XXX_Unmarshal(b []byte) error { @@ -1428,7 +1538,7 @@ func (m *Key2SegMsg) Reset() { *m = Key2SegMsg{} } func (m *Key2SegMsg) String() string { return proto.CompactTextString(m) } func (*Key2SegMsg) ProtoMessage() {} func (*Key2SegMsg) Descriptor() ([]byte, []int) { - return fileDescriptor_7eb37f6b80b23116, []int{18} + return fileDescriptor_7eb37f6b80b23116, []int{20} } func (m *Key2SegMsg) XXX_Unmarshal(b []byte) error { @@ -1476,7 +1586,7 @@ func (m *SegmentStatistics) Reset() { *m = SegmentStatistics{} } func (m *SegmentStatistics) String() string { return proto.CompactTextString(m) } func (*SegmentStatistics) ProtoMessage() {} func (*SegmentStatistics) Descriptor() ([]byte, []int) { - return fileDescriptor_7eb37f6b80b23116, []int{19} + return fileDescriptor_7eb37f6b80b23116, []int{21} } func (m *SegmentStatistics) XXX_Unmarshal(b []byte) error { @@ -1521,6 +1631,8 @@ func (m *SegmentStatistics) GetNumRows() int64 { func init() { proto.RegisterEnum("milvus.proto.internal.ReqType", ReqType_name, ReqType_value) proto.RegisterEnum("milvus.proto.internal.PeerRole", PeerRole_name, PeerRole_value) + proto.RegisterType((*IdRequest)(nil), "milvus.proto.internal.IdRequest") + proto.RegisterType((*IdResponse)(nil), "milvus.proto.internal.IdResponse") proto.RegisterType((*TsoRequest)(nil), "milvus.proto.internal.TsoRequest") proto.RegisterType((*TsoResponse)(nil), "milvus.proto.internal.TsoResponse") proto.RegisterType((*CreateCollectionRequest)(nil), "milvus.proto.internal.CreateCollectionRequest") @@ -1546,77 +1658,79 @@ func init() { func init() { proto.RegisterFile("internal_msg.proto", fileDescriptor_7eb37f6b80b23116) } var fileDescriptor_7eb37f6b80b23116 = []byte{ - // 1147 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x58, 0x49, 0x6f, 0x23, 0x45, - 0x14, 0x9e, 0xf6, 0xee, 0xe7, 0xd8, 0xe9, 0x54, 0x62, 0xe2, 0x19, 0x60, 0xc6, 0xf4, 0x20, 0x11, - 0x8d, 0x84, 0x23, 0x3c, 0x1c, 0x98, 0xeb, 0xc4, 0x87, 0x98, 0x51, 0xa2, 0xa8, 0x1d, 0x81, 0x84, - 0x84, 0x5a, 0xed, 0xee, 0x87, 0x5d, 0xea, 0x35, 0x55, 0xe5, 0x04, 0xe7, 0x0f, 0x70, 0x05, 0x71, - 0xe4, 0xc6, 0x99, 0x03, 0x3f, 0x83, 0xed, 0xce, 0x9f, 0x00, 0xc1, 0x48, 0xa0, 0xb9, 0xa2, 0xaa, - 0x6e, 0x2f, 0xed, 0x2c, 0x2c, 0x62, 0xa4, 0x48, 0xb9, 0xf5, 0x7b, 0xb5, 0xbd, 0xef, 0x7b, 0x8b, - 0xdf, 0x33, 0x10, 0x1a, 0x0a, 0x64, 0xa1, 0xed, 0x5b, 0x01, 0x1f, 0x75, 0x62, 0x16, 0x89, 0x88, - 0x34, 0x03, 0xea, 0x9f, 0x4e, 0x78, 0x22, 0x75, 0x66, 0x1b, 0xee, 0xad, 0x39, 0x51, 0x10, 0x44, - 0x61, 0xa2, 0xbe, 0xb7, 0xc1, 0x91, 0x9d, 0x52, 0x07, 0x17, 0xe7, 0x8c, 0x18, 0xe0, 0x98, 0x47, - 0x26, 0x9e, 0x4c, 0x90, 0x0b, 0xb2, 0x0d, 0xe5, 0x18, 0x91, 0x59, 0xd4, 0x6d, 0x69, 0x6d, 0x6d, - 0x27, 0x6f, 0x96, 0xa4, 0xd8, 0x77, 0xc9, 0x63, 0x28, 0xb0, 0xc8, 0xc7, 0x56, 0xae, 0xad, 0xed, - 0x34, 0xba, 0x0f, 0x3a, 0x97, 0xbe, 0xd6, 0x39, 0x42, 0x64, 0x66, 0xe4, 0xa3, 0xa9, 0x36, 0x93, - 0x2d, 0x28, 0x3a, 0xd1, 0x24, 0x14, 0xad, 0x7c, 0x5b, 0xdb, 0xa9, 0x9b, 0x89, 0x60, 0x9c, 0x42, - 0x4d, 0xbd, 0xc8, 0xe3, 0x28, 0xe4, 0x48, 0x1e, 0x43, 0x89, 0x0b, 0x5b, 0x4c, 0xb8, 0x7a, 0xb1, - 0xd6, 0x7d, 0x35, 0x7b, 0x77, 0x6a, 0xff, 0x40, 0x6d, 0x31, 0xd3, 0xad, 0xe4, 0x35, 0xa8, 0x0a, - 0x1a, 0x20, 0x17, 0x76, 0x10, 0x2b, 0x9b, 0x0a, 0xe6, 0x42, 0x71, 0xc5, 0xbb, 0x3f, 0x6b, 0xb0, - 0xbd, 0xc7, 0xd0, 0x16, 0xb8, 0x17, 0xf9, 0x3e, 0x3a, 0x82, 0x46, 0xe1, 0x0c, 0xf7, 0x13, 0xa8, - 0x30, 0x3c, 0xb1, 0xc4, 0x34, 0x46, 0x65, 0x46, 0xa3, 0x7b, 0xff, 0x0a, 0x88, 0x26, 0x9e, 0x1c, - 0x4f, 0x63, 0x34, 0xcb, 0x2c, 0xf9, 0x20, 0x4d, 0x28, 0xc9, 0xa3, 0xd4, 0x55, 0x76, 0xe4, 0xcd, - 0x22, 0xc3, 0x93, 0xbe, 0x9b, 0xb5, 0x30, 0xbf, 0x6a, 0xe1, 0x5d, 0xa8, 0xc4, 0x2c, 0xfa, 0x74, - 0x2a, 0x8f, 0x15, 0xd4, 0xb1, 0xb2, 0x92, 0xfb, 0x2e, 0x79, 0x07, 0x4a, 0xdc, 0x19, 0x63, 0x60, - 0xb7, 0x8a, 0x8a, 0x8f, 0xbb, 0x97, 0xf2, 0xf1, 0xd4, 0x8f, 0x86, 0x66, 0xba, 0xd1, 0x78, 0xae, - 0x41, 0xb3, 0xc7, 0xa2, 0xf8, 0x46, 0xe3, 0x3a, 0x80, 0x75, 0x67, 0x6e, 0x9f, 0x15, 0xda, 0x01, - 0xa6, 0x00, 0xdf, 0xcc, 0x5a, 0x94, 0x86, 0x68, 0x67, 0x01, 0xe6, 0xd0, 0x0e, 0xd0, 0x6c, 0x38, - 0x19, 0xd9, 0xf8, 0x5d, 0x83, 0xad, 0x7d, 0x9b, 0xdf, 0x26, 0xc8, 0x7f, 0x6a, 0x70, 0xb7, 0x87, - 0xdc, 0x61, 0x74, 0x88, 0xb7, 0x09, 0xf7, 0xd7, 0x1a, 0x34, 0x07, 0xe3, 0xe8, 0xec, 0x26, 0x63, - 0x36, 0x7e, 0xd3, 0xe0, 0x95, 0xa4, 0xba, 0x1c, 0xd9, 0x4c, 0xd0, 0x1b, 0xea, 0x99, 0xf7, 0xa1, - 0x11, 0xcf, 0xcc, 0x5b, 0x76, 0xcc, 0xc3, 0xcb, 0x1d, 0x33, 0x87, 0xa2, 0xfc, 0x52, 0x8f, 0x97, - 0x45, 0xe3, 0x57, 0x0d, 0xb6, 0x64, 0xd5, 0xb9, 0x2d, 0x78, 0x7f, 0xd1, 0x60, 0x73, 0xdf, 0xe6, - 0xb7, 0x05, 0xee, 0x73, 0x0d, 0x5a, 0xb3, 0x6a, 0x73, 0x5b, 0x30, 0xcb, 0x1f, 0x15, 0x59, 0x69, - 0x6e, 0x32, 0xde, 0xff, 0xb9, 0xb8, 0xbe, 0xc8, 0x41, 0xbd, 0x1f, 0x72, 0x64, 0xe2, 0xe5, 0x61, - 0x7d, 0xeb, 0xa2, 0xc9, 0x12, 0x71, 0x75, 0xd5, 0x18, 0xf2, 0x10, 0x16, 0x0e, 0xb1, 0x84, 0x3d, - 0x52, 0xd8, 0xab, 0xe6, 0xda, 0x5c, 0x79, 0x6c, 0x8f, 0xc8, 0xeb, 0x00, 0x1c, 0x47, 0x01, 0x86, - 0x42, 0x3e, 0x54, 0x54, 0x0f, 0x55, 0x53, 0x4d, 0xdf, 0x95, 0xcb, 0xce, 0xd8, 0x0e, 0x43, 0xf4, - 0xe5, 0x72, 0x29, 0x59, 0x4e, 0x35, 0x7d, 0x37, 0xc3, 0x6c, 0x39, 0xcb, 0xec, 0x7d, 0x80, 0xb9, - 0x07, 0x78, 0xab, 0xd2, 0xce, 0xef, 0x14, 0xcc, 0x25, 0x8d, 0x6c, 0x8e, 0x59, 0x74, 0x66, 0x51, - 0x97, 0xb7, 0xaa, 0xed, 0xbc, 0x6c, 0x8e, 0x59, 0x74, 0xd6, 0x77, 0x39, 0x79, 0x17, 0x2a, 0x72, - 0xc1, 0xb5, 0x85, 0xdd, 0x82, 0x76, 0xfe, 0xfa, 0xa6, 0x4d, 0xde, 0xd1, 0xb3, 0x85, 0x6d, 0x7c, - 0x96, 0x83, 0x7a, 0x0f, 0x7d, 0x14, 0x78, 0x03, 0x98, 0xcf, 0xb2, 0x56, 0xb8, 0x8e, 0xb5, 0xe2, - 0x75, 0xac, 0x95, 0x2e, 0xb0, 0xf6, 0x06, 0xac, 0xc5, 0x8c, 0x06, 0x36, 0x9b, 0x5a, 0x1e, 0x4e, - 0x79, 0xab, 0xac, 0xa8, 0xab, 0xa5, 0xba, 0x67, 0x38, 0xe5, 0xc6, 0x0b, 0x0d, 0xea, 0x03, 0xb4, - 0x99, 0x33, 0x7e, 0x79, 0x4c, 0x2c, 0x23, 0xc8, 0x67, 0x11, 0x64, 0x52, 0xb1, 0xb0, 0x9a, 0x8a, - 0x8f, 0x60, 0x83, 0x21, 0x9f, 0xf8, 0xc2, 0x5a, 0x22, 0x28, 0xe1, 0x60, 0x3d, 0x59, 0xd8, 0x9b, - 0xd3, 0xb4, 0x0b, 0xc5, 0x93, 0x09, 0xb2, 0xa9, 0x0a, 0xbb, 0x6b, 0xa3, 0x20, 0xd9, 0x67, 0x7c, - 0x99, 0x83, 0xb5, 0x19, 0x72, 0x79, 0xd5, 0x7f, 0x9b, 0x86, 0xfe, 0x3d, 0x64, 0x03, 0xea, 0xca, - 0x00, 0x2b, 0x8c, 0x5c, 0x5c, 0x78, 0xbc, 0xa6, 0x94, 0x87, 0x91, 0x8b, 0xab, 0xb4, 0x14, 0xff, - 0x11, 0x2d, 0xa5, 0xcb, 0x69, 0xe9, 0x40, 0x61, 0x4c, 0x45, 0xe2, 0xfa, 0x5a, 0xf7, 0xde, 0xe5, - 0x75, 0x6a, 0x9f, 0x0a, 0x6e, 0xaa, 0x7d, 0x46, 0x0f, 0x6a, 0xc7, 0x34, 0xc0, 0xc1, 0x34, 0x74, - 0x0e, 0xf8, 0xe8, 0xea, 0xa1, 0xf4, 0xda, 0x29, 0xd0, 0xf8, 0x4a, 0x83, 0xf2, 0x33, 0x9c, 0x76, - 0x07, 0x38, 0x52, 0x0c, 0xa9, 0xd4, 0x4d, 0x6f, 0x28, 0xaa, 0xcc, 0x25, 0x0f, 0xa0, 0xb6, 0x14, - 0x9b, 0x29, 0x7b, 0xb0, 0x08, 0xcd, 0xbf, 0xaf, 0xd2, 0x94, 0x5b, 0xa7, 0xb6, 0x9f, 0x12, 0x58, - 0x31, 0xcb, 0x94, 0x7f, 0x20, 0x45, 0x79, 0xf3, 0xa2, 0x48, 0xf1, 0x56, 0x51, 0x05, 0x3d, 0xcc, - 0xab, 0x14, 0x37, 0x3e, 0x06, 0x48, 0x8d, 0x93, 0x10, 0x17, 0x1e, 0xd4, 0x96, 0x3d, 0xf8, 0x1e, - 0x94, 0x3d, 0x9c, 0x76, 0x39, 0x8e, 0x5a, 0x39, 0xc5, 0xdd, 0x55, 0x59, 0x90, 0x5e, 0x65, 0xce, - 0xb6, 0x1b, 0x21, 0x6c, 0x0c, 0x92, 0xc7, 0x64, 0xac, 0x50, 0x2e, 0xa8, 0xc3, 0x57, 0x2a, 0xa7, - 0xb6, 0x5a, 0x39, 0x1f, 0x40, 0x2d, 0xc0, 0x20, 0x62, 0x53, 0x8b, 0xd3, 0x73, 0x9c, 0xb1, 0x91, - 0xa8, 0x06, 0xf4, 0x1c, 0x25, 0xde, 0x70, 0x12, 0x58, 0x2c, 0x3a, 0xe3, 0xb3, 0x80, 0x0a, 0x27, - 0x81, 0x19, 0x9d, 0xf1, 0x47, 0xdf, 0xe4, 0xa0, 0x9c, 0xa6, 0x22, 0xa9, 0x42, 0xd1, 0x3b, 0x8c, - 0x42, 0xd4, 0xef, 0x90, 0x26, 0x6c, 0x78, 0xab, 0x33, 0xb7, 0xee, 0x92, 0x4d, 0x58, 0xf7, 0xb2, - 0x03, 0xab, 0x8e, 0x84, 0x40, 0xc3, 0xcb, 0x4c, 0x74, 0xfa, 0x27, 0x64, 0x1b, 0x36, 0xbd, 0x8b, - 0x23, 0x8f, 0x3e, 0x22, 0x5b, 0xa0, 0x7b, 0xd9, 0x99, 0x80, 0xeb, 0x63, 0xd2, 0x04, 0xdd, 0x5b, - 0x69, 0xc2, 0xf5, 0xef, 0x34, 0xb2, 0x09, 0x0d, 0x2f, 0xd3, 0xa9, 0xea, 0xdf, 0x6b, 0x84, 0x40, - 0xdd, 0x5b, 0x6e, 0xe7, 0xf4, 0x1f, 0x34, 0xb2, 0x0d, 0xc4, 0xbb, 0xd0, 0xf3, 0xe8, 0x3f, 0x6a, - 0x64, 0x0b, 0xd6, 0xbd, 0x4c, 0x63, 0xc0, 0xf5, 0x9f, 0x34, 0xb2, 0x06, 0x65, 0x2f, 0xf9, 0xed, - 0xd4, 0x3f, 0xcf, 0x2b, 0x29, 0xa9, 0xe7, 0xfa, 0x17, 0x89, 0x94, 0x64, 0xb6, 0xfe, 0x47, 0x9e, - 0x34, 0xa0, 0xea, 0xc9, 0x90, 0x3e, 0xa6, 0x8e, 0xa7, 0x7f, 0x5b, 0x7d, 0xf4, 0x04, 0x2a, 0xb3, - 0xff, 0x4a, 0x08, 0x40, 0xe9, 0xc0, 0xe6, 0x02, 0x99, 0x7e, 0x47, 0x7e, 0x9b, 0x68, 0xbb, 0xc8, - 0x74, 0x4d, 0x7e, 0x7f, 0xc8, 0xa8, 0xd4, 0xe7, 0x24, 0xa5, 0x47, 0x32, 0x75, 0xf5, 0xfc, 0xd3, - 0xde, 0x47, 0x4f, 0x47, 0x54, 0x8c, 0x27, 0x43, 0x59, 0x0c, 0x76, 0xcf, 0xa9, 0xef, 0xd3, 0x73, - 0x81, 0xce, 0x78, 0x37, 0x89, 0x8c, 0xb7, 0x5d, 0xca, 0x05, 0xa3, 0xc3, 0x89, 0x40, 0x77, 0x77, - 0x16, 0x1f, 0xbb, 0x2a, 0x5c, 0xe6, 0x62, 0x3c, 0x1c, 0x96, 0x94, 0xe6, 0xf1, 0x5f, 0x01, 0x00, - 0x00, 0xff, 0xff, 0x7a, 0xb3, 0x5a, 0xe2, 0x4b, 0x12, 0x00, 0x00, + // 1171 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x58, 0x4b, 0x6f, 0x1b, 0x55, + 0x14, 0xee, 0xf8, 0x19, 0x1f, 0xc7, 0xce, 0xe4, 0x26, 0x21, 0x6e, 0x81, 0xd6, 0x4c, 0x91, 0x88, + 0x2a, 0xe1, 0x08, 0x97, 0x05, 0xdd, 0xb6, 0x5e, 0xd4, 0x54, 0xad, 0xaa, 0x71, 0x04, 0x12, 0x12, + 0x1a, 0x8d, 0x67, 0x0e, 0xf6, 0xd5, 0x3c, 0xee, 0xe4, 0xde, 0xeb, 0x04, 0xe7, 0x0f, 0xb0, 0x05, + 0xb1, 0x64, 0xc7, 0x9a, 0x05, 0x3f, 0x83, 0xd7, 0x9e, 0x3f, 0x01, 0x82, 0x4a, 0xa0, 0x6e, 0xd1, + 0xbd, 0x33, 0x7e, 0x8c, 0xf3, 0xe0, 0x59, 0x14, 0x29, 0xbb, 0x39, 0xe7, 0x3e, 0xce, 0xf9, 0xbe, + 0xf3, 0xf0, 0x3d, 0x06, 0x42, 0x63, 0x89, 0x3c, 0x76, 0x43, 0x27, 0x12, 0xa3, 0x4e, 0xc2, 0x99, + 0x64, 0x64, 0x27, 0xa2, 0xe1, 0xd1, 0x44, 0xa4, 0x52, 0x67, 0xb6, 0xe1, 0xc6, 0xba, 0xc7, 0xa2, + 0x88, 0xc5, 0xa9, 0xfa, 0xc6, 0xa6, 0x40, 0x7e, 0x44, 0x3d, 0x5c, 0x9c, 0xb3, 0x18, 0xd4, 0xfa, + 0xbe, 0x8d, 0x87, 0x13, 0x14, 0x92, 0xec, 0x42, 0x35, 0x41, 0xe4, 0x0e, 0xf5, 0x5b, 0x46, 0xdb, + 0xd8, 0x2b, 0xda, 0x15, 0x25, 0xf6, 0x7d, 0x72, 0x17, 0x4a, 0x9c, 0x85, 0xd8, 0x2a, 0xb4, 0x8d, + 0xbd, 0x66, 0xf7, 0x56, 0xe7, 0x4c, 0x63, 0x9d, 0xa7, 0x88, 0xdc, 0x66, 0x21, 0xda, 0x7a, 0x33, + 0xd9, 0x86, 0xb2, 0xc7, 0x26, 0xb1, 0x6c, 0x15, 0xdb, 0xc6, 0x5e, 0xc3, 0x4e, 0x05, 0x6b, 0x04, + 0xa0, 0x0c, 0x8a, 0x84, 0xc5, 0x02, 0xc9, 0x5d, 0xa8, 0x08, 0xe9, 0xca, 0x89, 0xd0, 0x06, 0xeb, + 0xdd, 0x97, 0xf3, 0x57, 0x67, 0xde, 0x0f, 0xf4, 0x16, 0x3b, 0xdb, 0x4a, 0x9a, 0x50, 0xa0, 0xbe, + 0xf6, 0xa5, 0x68, 0x17, 0xa8, 0x7f, 0x8e, 0xa1, 0x04, 0xe0, 0x40, 0xb0, 0xff, 0x13, 0xda, 0x11, + 0xd4, 0xb5, 0xc5, 0x7f, 0x83, 0xed, 0x15, 0xa8, 0x49, 0x1a, 0xa1, 0x90, 0x6e, 0x94, 0x68, 0x9f, + 0x4a, 0xf6, 0x42, 0x71, 0x8e, 0xdd, 0x1f, 0x0d, 0xd8, 0x7d, 0xc0, 0xd1, 0x95, 0xf8, 0x80, 0x85, + 0x21, 0x7a, 0x92, 0xb2, 0x78, 0x86, 0xfb, 0x1e, 0xac, 0x71, 0x3c, 0x74, 0xe4, 0x34, 0x41, 0xed, + 0x46, 0xb3, 0x7b, 0xf3, 0x1c, 0x88, 0x36, 0x1e, 0x1e, 0x4c, 0x13, 0xb4, 0xab, 0x3c, 0xfd, 0x20, + 0x3b, 0x50, 0x51, 0x47, 0xe7, 0x54, 0x97, 0x39, 0x1e, 0xf6, 0xfd, 0xbc, 0x87, 0xc5, 0x55, 0x0f, + 0xaf, 0xc3, 0x5a, 0xc2, 0xd9, 0xc7, 0x53, 0x75, 0xac, 0xa4, 0x8f, 0x55, 0xb5, 0xdc, 0xf7, 0xc9, + 0x5b, 0x50, 0x11, 0xde, 0x18, 0x23, 0xb7, 0x55, 0xd6, 0x7c, 0x5c, 0x3f, 0x93, 0x8f, 0xfb, 0x21, + 0x1b, 0xda, 0xd9, 0x46, 0xeb, 0x99, 0x01, 0x3b, 0x3d, 0xce, 0x92, 0x4b, 0x8d, 0xeb, 0x31, 0x6c, + 0x78, 0x73, 0xff, 0x9c, 0xd8, 0x8d, 0x30, 0x03, 0xf8, 0x7a, 0xde, 0xa3, 0xac, 0xf8, 0x3a, 0x0b, + 0x30, 0x4f, 0xdc, 0x08, 0xed, 0xa6, 0x97, 0x93, 0xad, 0x5f, 0x0d, 0xd8, 0x7e, 0xe8, 0x8a, 0xab, + 0x04, 0xf9, 0x77, 0x03, 0xae, 0xf7, 0x50, 0x78, 0x9c, 0x0e, 0xf1, 0x2a, 0xe1, 0xfe, 0xd2, 0x80, + 0x9d, 0xc1, 0x98, 0x1d, 0x5f, 0x66, 0xcc, 0xd6, 0x2f, 0x06, 0xbc, 0x94, 0x76, 0x97, 0xa7, 0x2e, + 0x97, 0xf4, 0x92, 0x46, 0xe6, 0x5d, 0x68, 0x26, 0x33, 0xf7, 0x96, 0x03, 0x73, 0xfb, 0xec, 0xc0, + 0xcc, 0xa1, 0xe8, 0xb8, 0x34, 0x92, 0x65, 0xd1, 0xfa, 0xd9, 0x80, 0x6d, 0xd5, 0x75, 0xae, 0x0a, + 0xde, 0x9f, 0x0c, 0xd8, 0x7a, 0xe8, 0x8a, 0xab, 0x02, 0xf7, 0x99, 0x01, 0xad, 0x59, 0xb7, 0xb9, + 0x2a, 0x98, 0xd5, 0x8f, 0x8a, 0xea, 0x34, 0x97, 0x19, 0xef, 0x7f, 0xdc, 0x5c, 0x9f, 0x17, 0xa0, + 0xd1, 0x8f, 0x05, 0x72, 0xf9, 0xe2, 0xb0, 0xbe, 0x71, 0xda, 0x65, 0x85, 0xb8, 0xb6, 0xea, 0x0c, + 0xb9, 0x0d, 0x8b, 0x80, 0x38, 0xd2, 0x1d, 0x69, 0xec, 0x35, 0x7b, 0x7d, 0xae, 0x3c, 0x70, 0x47, + 0xe4, 0x55, 0x00, 0x81, 0xa3, 0x08, 0x63, 0xa9, 0x0c, 0x95, 0xb5, 0xa1, 0x5a, 0xa6, 0xe9, 0xfb, + 0x6a, 0xd9, 0x1b, 0xbb, 0x71, 0x8c, 0xa1, 0x5a, 0xae, 0xa4, 0xcb, 0x99, 0xa6, 0xef, 0xe7, 0x98, + 0xad, 0xe6, 0x99, 0xbd, 0x09, 0x30, 0x8f, 0x80, 0x68, 0xad, 0xb5, 0x8b, 0x7b, 0x25, 0x7b, 0x49, + 0xa3, 0x1e, 0xc7, 0x9c, 0x1d, 0x3b, 0xd4, 0x17, 0xad, 0x5a, 0xbb, 0xa8, 0x1e, 0xc7, 0x9c, 0x1d, + 0xf7, 0x7d, 0x41, 0xde, 0x86, 0x35, 0xb5, 0xe0, 0xbb, 0xd2, 0x6d, 0x41, 0xbb, 0x78, 0xf1, 0xa3, + 0x4d, 0xdd, 0xd1, 0x73, 0xa5, 0x6b, 0x7d, 0x52, 0x80, 0x46, 0x0f, 0x43, 0x94, 0x78, 0x09, 0x98, + 0xcf, 0xb3, 0x56, 0xba, 0x88, 0xb5, 0xf2, 0x45, 0xac, 0x55, 0x4e, 0xb1, 0xf6, 0x1a, 0xac, 0x27, + 0x9c, 0x46, 0x2e, 0x9f, 0x3a, 0x01, 0x4e, 0x45, 0xab, 0xaa, 0xa9, 0xab, 0x67, 0xba, 0x47, 0x38, + 0x15, 0xd6, 0x73, 0x03, 0x1a, 0x03, 0x74, 0xb9, 0x37, 0x7e, 0x71, 0x4c, 0x2c, 0x23, 0x28, 0xe6, + 0x11, 0xe4, 0x4a, 0xb1, 0xb4, 0x5a, 0x8a, 0x77, 0x60, 0x93, 0xa3, 0x98, 0x84, 0xd2, 0x59, 0x22, + 0x28, 0xe5, 0x60, 0x23, 0x5d, 0x78, 0x30, 0xa7, 0x69, 0x1f, 0xca, 0x87, 0x13, 0xe4, 0x53, 0x9d, + 0x76, 0x17, 0x66, 0x41, 0xba, 0xcf, 0xfa, 0xbc, 0x00, 0xeb, 0x33, 0xe4, 0xea, 0xaa, 0x7f, 0x36, + 0x0d, 0xfd, 0x7d, 0xc8, 0x16, 0x34, 0xb4, 0x03, 0x4e, 0xcc, 0x7c, 0x5c, 0x44, 0xbc, 0xae, 0x95, + 0x4f, 0x98, 0x8f, 0xab, 0xb4, 0x94, 0xff, 0x12, 0x2d, 0x95, 0xb3, 0x69, 0xe9, 0x40, 0x69, 0x4c, + 0x65, 0x1a, 0xfa, 0x7a, 0xf7, 0xc6, 0xd9, 0x7d, 0xea, 0x21, 0x95, 0xc2, 0xd6, 0xfb, 0xac, 0x1e, + 0xd4, 0x0f, 0x68, 0x84, 0x83, 0x69, 0xec, 0x3d, 0x16, 0xa3, 0xf3, 0x87, 0xd2, 0x0b, 0xa7, 0x40, + 0xeb, 0x0b, 0x03, 0xaa, 0x8f, 0x70, 0xda, 0x1d, 0xe0, 0x48, 0x33, 0xa4, 0x4b, 0x37, 0xbb, 0xa1, + 0xac, 0x2b, 0x97, 0xdc, 0x82, 0xfa, 0x52, 0x6e, 0x66, 0xec, 0xc1, 0x22, 0x35, 0xff, 0xbc, 0x4b, + 0x53, 0xe1, 0x1c, 0xb9, 0x61, 0x46, 0xe0, 0x9a, 0x5d, 0xa5, 0xe2, 0x3d, 0x25, 0xaa, 0x9b, 0x17, + 0x4d, 0x4a, 0xb4, 0xca, 0x3a, 0xe9, 0x61, 0xde, 0xa5, 0x84, 0xf5, 0x21, 0x40, 0xe6, 0x9c, 0x82, + 0xb8, 0x88, 0xa0, 0xb1, 0x1c, 0xc1, 0x77, 0xa0, 0x1a, 0xe0, 0xb4, 0x2b, 0x70, 0xd4, 0x2a, 0x68, + 0xee, 0xce, 0xab, 0x82, 0xec, 0x2a, 0x7b, 0xb6, 0xdd, 0x8a, 0x61, 0x73, 0x90, 0x1a, 0x53, 0xb9, + 0x42, 0x85, 0xa4, 0x9e, 0x58, 0xe9, 0x9c, 0xc6, 0x6a, 0xe7, 0xbc, 0x05, 0xf5, 0x08, 0x23, 0xc6, + 0xa7, 0x8e, 0xa0, 0x27, 0x38, 0x63, 0x23, 0x55, 0x0d, 0xe8, 0x09, 0x2a, 0xbc, 0xf1, 0x24, 0x72, + 0x38, 0x3b, 0x16, 0xb3, 0x84, 0x8a, 0x27, 0x91, 0xcd, 0x8e, 0xc5, 0x9d, 0xaf, 0x0a, 0x50, 0xcd, + 0x4a, 0x91, 0xd4, 0xa0, 0x1c, 0x3c, 0x61, 0x31, 0x9a, 0xd7, 0xc8, 0x0e, 0x6c, 0x06, 0xab, 0x33, + 0xb7, 0xe9, 0x93, 0x2d, 0xd8, 0x08, 0xf2, 0x03, 0xab, 0x89, 0x84, 0x40, 0x33, 0xc8, 0x4d, 0x74, + 0xe6, 0x47, 0x64, 0x17, 0xb6, 0x82, 0xd3, 0x23, 0x8f, 0x39, 0x22, 0xdb, 0x60, 0x06, 0xf9, 0x99, + 0x40, 0x98, 0x63, 0xb2, 0x03, 0x66, 0xb0, 0xf2, 0x08, 0x37, 0xbf, 0x31, 0xc8, 0x16, 0x34, 0x83, + 0xdc, 0x4b, 0xd5, 0xfc, 0xd6, 0x20, 0x04, 0x1a, 0xc1, 0xf2, 0x73, 0xce, 0xfc, 0xce, 0x20, 0xbb, + 0x40, 0x82, 0x53, 0x6f, 0x1e, 0xf3, 0x7b, 0x83, 0x6c, 0xc3, 0x46, 0x90, 0x7b, 0x18, 0x08, 0xf3, + 0x07, 0x83, 0xac, 0x43, 0x35, 0x48, 0x7f, 0x3b, 0xcd, 0x4f, 0x8b, 0x5a, 0x4a, 0xfb, 0xb9, 0xf9, + 0x59, 0x2a, 0xa5, 0x95, 0x6d, 0xfe, 0x56, 0x24, 0x4d, 0xa8, 0x05, 0x2a, 0xa5, 0x0f, 0xa8, 0x17, + 0x98, 0x5f, 0xd7, 0xee, 0xdc, 0x83, 0xb5, 0xd9, 0x7f, 0x25, 0x04, 0xa0, 0xf2, 0xd8, 0x15, 0x12, + 0xb9, 0x79, 0x4d, 0x7d, 0xdb, 0xe8, 0xfa, 0xc8, 0x4d, 0x43, 0x7d, 0xbf, 0xcf, 0xa9, 0xd2, 0x17, + 0x14, 0xa5, 0x4f, 0x55, 0xe9, 0x9a, 0xc5, 0xfb, 0xbd, 0x0f, 0xee, 0x8f, 0xa8, 0x1c, 0x4f, 0x86, + 0xaa, 0x19, 0xec, 0x9f, 0xd0, 0x30, 0xa4, 0x27, 0x12, 0xbd, 0xf1, 0x7e, 0x9a, 0x19, 0x6f, 0xfa, + 0x54, 0x48, 0x4e, 0x87, 0x13, 0x89, 0xfe, 0xfe, 0x2c, 0x3f, 0xf6, 0x75, 0xba, 0xcc, 0xc5, 0x64, + 0x38, 0xac, 0x68, 0xcd, 0xdd, 0x3f, 0x02, 0x00, 0x00, 0xff, 0xff, 0x2d, 0x19, 0x41, 0x9d, 0x25, + 0x13, 0x00, 0x00, } diff --git a/internal/proto/master.proto b/internal/proto/master.proto index 834dcbc19f..83feed58e2 100644 --- a/internal/proto/master.proto +++ b/internal/proto/master.proto @@ -89,5 +89,6 @@ service Master { rpc ShowPartitions(internal.ShowPartitionRequest) returns (service.StringListResponse) {} - rpc Tso(stream internal.TsoRequest) returns (stream internal.TsoResponse) {} + rpc AllocTimestamp(internal.TsoRequest) returns (internal.TsoResponse) {} + rpc AllocId(internal.IdRequest) returns (internal.IdResponse) {} } diff --git a/internal/proto/masterpb/master.pb.go b/internal/proto/masterpb/master.pb.go index 70f58cd90a..4e820efc36 100644 --- a/internal/proto/masterpb/master.pb.go +++ b/internal/proto/masterpb/master.pb.go @@ -30,33 +30,34 @@ const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package func init() { proto.RegisterFile("master.proto", fileDescriptor_f9c348dec43a6705) } var fileDescriptor_f9c348dec43a6705 = []byte{ - // 409 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x94, 0xcd, 0x6e, 0xe2, 0x30, - 0x14, 0x85, 0x41, 0x23, 0xa1, 0x91, 0xc5, 0xcf, 0xe0, 0xd9, 0x31, 0xab, 0xc9, 0x0a, 0xc1, 0x90, - 0xa0, 0xe9, 0x0b, 0x54, 0xc0, 0x82, 0x45, 0x2b, 0x55, 0xc0, 0xaa, 0x55, 0x45, 0x9d, 0x60, 0x81, - 0xd5, 0x24, 0x4e, 0x7d, 0x6f, 0xa8, 0xc4, 0xfb, 0xf6, 0x3d, 0xaa, 0x24, 0x84, 0xc4, 0x05, 0x53, - 0xda, 0x1d, 0xb6, 0x8f, 0xbf, 0x83, 0xef, 0x39, 0x0a, 0xa9, 0x07, 0x0c, 0x90, 0x2b, 0x3b, 0x52, - 0x12, 0x25, 0xfd, 0x1d, 0x08, 0x7f, 0x1b, 0x43, 0xb6, 0xb2, 0xb3, 0xa3, 0x4e, 0xdd, 0x93, 0x41, - 0x20, 0xc3, 0x6c, 0xb3, 0x43, 0x45, 0x88, 0x5c, 0x85, 0xcc, 0x5f, 0x06, 0xb0, 0xde, 0xef, 0xb5, - 0x81, 0xab, 0xad, 0xf0, 0x78, 0xb1, 0xf5, 0xff, 0xed, 0x27, 0xa9, 0xdd, 0xa6, 0xf7, 0x29, 0x23, - 0xbf, 0xc6, 0x8a, 0x33, 0xe4, 0x63, 0xe9, 0xfb, 0xdc, 0x43, 0x21, 0x43, 0x6a, 0xdb, 0x9a, 0x53, - 0xce, 0xb4, 0x3f, 0x0a, 0x67, 0xfc, 0x25, 0xe6, 0x80, 0x9d, 0x3f, 0xba, 0x7e, 0xff, 0x8f, 0xe6, - 0xc8, 0x30, 0x06, 0xab, 0x42, 0x1f, 0x49, 0x73, 0xa2, 0x64, 0x54, 0x32, 0xf8, 0x67, 0x30, 0xd0, - 0x65, 0x17, 0xe2, 0x5d, 0xd2, 0x98, 0x32, 0x28, 0xd1, 0xfb, 0x06, 0xba, 0xa6, 0xca, 0xe1, 0x96, - 0x2e, 0xde, 0xcf, 0xca, 0x1e, 0x49, 0xe9, 0xcf, 0x38, 0x44, 0x32, 0x04, 0x6e, 0x55, 0x68, 0x4c, - 0xe8, 0x84, 0x83, 0xa7, 0x84, 0x5b, 0x9e, 0xd3, 0xd0, 0xf4, 0x8c, 0x23, 0x69, 0xee, 0xd6, 0x3f, - 0xed, 0x56, 0x08, 0xb3, 0xab, 0x51, 0xf2, 0xd3, 0xaa, 0xd0, 0x67, 0xd2, 0x9a, 0x6f, 0xe4, 0x6b, - 0x71, 0x0c, 0xc6, 0xd1, 0xe9, 0xba, 0xdc, 0xaf, 0x7b, 0xda, 0x6f, 0x8e, 0x4a, 0x84, 0xeb, 0x1b, - 0x01, 0x58, 0x7a, 0xe3, 0x92, 0xb4, 0xb2, 0x80, 0xef, 0x98, 0x42, 0x91, 0x3e, 0x70, 0x70, 0xb6, - 0x08, 0x07, 0xdd, 0x85, 0x41, 0x3d, 0x90, 0x46, 0x12, 0x70, 0x81, 0xef, 0x9f, 0xa9, 0xc1, 0x57, - 0xe1, 0x4f, 0xa4, 0x3e, 0x65, 0x50, 0xb0, 0x7b, 0xe6, 0x12, 0x1c, 0xa1, 0x2f, 0xeb, 0x80, 0x22, - 0xed, 0x3c, 0xd8, 0xc2, 0xc6, 0xf9, 0xa4, 0x02, 0x47, 0x5e, 0xbd, 0xd3, 0x5e, 0x07, 0x9d, 0x5e, - 0x00, 0x41, 0x9a, 0x49, 0xb0, 0x87, 0x53, 0x30, 0xce, 0x4c, 0x93, 0x7d, 0x27, 0xfe, 0x19, 0xf9, - 0xb1, 0x00, 0x49, 0xff, 0x1a, 0xf8, 0x0b, 0x90, 0x86, 0x71, 0xe9, 0x92, 0x9c, 0xd7, 0xad, 0x0e, - 0xab, 0xa3, 0xd1, 0xfd, 0xf5, 0x5a, 0xe0, 0x26, 0x76, 0x93, 0xb8, 0x9c, 0x9d, 0xf0, 0x7d, 0xb1, - 0x43, 0xee, 0x6d, 0x9c, 0x0c, 0x30, 0x58, 0x09, 0x40, 0x25, 0xdc, 0x18, 0xf9, 0xca, 0xc9, 0x31, - 0x4e, 0x4a, 0x75, 0xb2, 0xcf, 0x5b, 0xe4, 0xba, 0xb5, 0x74, 0x7d, 0xf5, 0x1e, 0x00, 0x00, 0xff, - 0xff, 0x47, 0x62, 0x7a, 0xee, 0x0c, 0x05, 0x00, 0x00, + // 432 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x94, 0x51, 0xaf, 0xd2, 0x30, + 0x14, 0xc7, 0x79, 0xba, 0xc6, 0x86, 0xcb, 0xf5, 0xd6, 0x37, 0x7c, 0xf1, 0xee, 0xc9, 0x80, 0x6c, + 0x46, 0xbf, 0x80, 0x02, 0x0f, 0x90, 0x68, 0x42, 0x80, 0x17, 0x35, 0x06, 0xbb, 0xad, 0x81, 0xc6, + 0x6e, 0x9d, 0x3d, 0x67, 0x98, 0xf0, 0xe1, 0xfc, 0x6c, 0x66, 0x1b, 0xdd, 0x56, 0xa1, 0x88, 0xf7, + 0x8d, 0xb6, 0xff, 0xf3, 0xfb, 0x73, 0xce, 0xf9, 0x67, 0xa4, 0x9b, 0x30, 0x40, 0xae, 0xfd, 0x4c, + 0x2b, 0x54, 0xf4, 0x79, 0x22, 0xe4, 0x3e, 0x87, 0xea, 0xe4, 0x57, 0x4f, 0xfd, 0x6e, 0xa4, 0x92, + 0x44, 0xa5, 0xd5, 0x65, 0x9f, 0x8a, 0x14, 0xb9, 0x4e, 0x99, 0xdc, 0x24, 0xb0, 0x3d, 0xde, 0xdd, + 0x03, 0xd7, 0x7b, 0x11, 0xf1, 0xe6, 0xea, 0xed, 0xef, 0xa7, 0xe4, 0xe6, 0x53, 0x59, 0x4f, 0x19, + 0x79, 0x36, 0xd1, 0x9c, 0x21, 0x9f, 0x28, 0x29, 0x79, 0x84, 0x42, 0xa5, 0xd4, 0xf7, 0x2d, 0x27, + 0xc3, 0xf4, 0xff, 0x16, 0x2e, 0xf9, 0xcf, 0x9c, 0x03, 0xf6, 0x5f, 0xd8, 0xfa, 0xe3, 0x3f, 0x5a, + 0x21, 0xc3, 0x1c, 0xbc, 0x0e, 0xfd, 0x46, 0x7a, 0x53, 0xad, 0xb2, 0x96, 0xc1, 0x6b, 0x87, 0x81, + 0x2d, 0xbb, 0x12, 0x1f, 0x92, 0xdb, 0x19, 0x83, 0x16, 0x7d, 0xe8, 0xa0, 0x5b, 0x2a, 0x03, 0xf7, + 0x6c, 0xf1, 0x71, 0x56, 0xfe, 0x58, 0x29, 0xb9, 0xe4, 0x90, 0xa9, 0x14, 0xb8, 0xd7, 0xa1, 0x39, + 0xa1, 0x53, 0x0e, 0x91, 0x16, 0x61, 0x7b, 0x4e, 0x6f, 0x5c, 0x6d, 0x9c, 0x48, 0x8d, 0xdb, 0xf0, + 0xbc, 0x5b, 0x23, 0xac, 0x4a, 0xb3, 0xe2, 0xa7, 0xd7, 0xa1, 0x3f, 0xc8, 0xdd, 0x6a, 0xa7, 0x7e, + 0x35, 0xcf, 0xe0, 0x1c, 0x9d, 0xad, 0x33, 0x7e, 0xaf, 0xce, 0xfb, 0xad, 0x50, 0x8b, 0x74, 0xfb, + 0x51, 0x00, 0xb6, 0x7a, 0xdc, 0x90, 0xbb, 0x6a, 0xc1, 0x0b, 0xa6, 0x51, 0x94, 0x0d, 0x8e, 0x2e, + 0x06, 0xa1, 0xd6, 0x5d, 0xb9, 0xa8, 0xaf, 0xe4, 0xb6, 0x58, 0x70, 0x83, 0x1f, 0x5e, 0x88, 0xc1, + 0xff, 0xc2, 0xbf, 0x93, 0xee, 0x8c, 0x41, 0xc3, 0x1e, 0xb8, 0x43, 0x70, 0x82, 0xbe, 0x2e, 0x03, + 0x9a, 0xdc, 0x9b, 0xc5, 0x36, 0x36, 0xc1, 0x3f, 0x22, 0x70, 0xe2, 0x35, 0x38, 0xef, 0x55, 0xeb, + 0xec, 0x00, 0x08, 0xd2, 0x2b, 0x16, 0x5b, 0xbf, 0x82, 0x73, 0x66, 0x96, 0xec, 0x31, 0xeb, 0xff, + 0x4c, 0x7a, 0x1f, 0xa4, 0x54, 0xd1, 0x5a, 0x24, 0x1c, 0x90, 0x25, 0x19, 0x7d, 0x70, 0x58, 0xad, + 0x41, 0x39, 0x26, 0x67, 0x4b, 0x6a, 0xf4, 0x82, 0x3c, 0x29, 0xd1, 0xf3, 0x98, 0xbe, 0x74, 0x14, + 0xcc, 0x63, 0x83, 0x7c, 0xb8, 0xa0, 0x30, 0xc4, 0xf1, 0xf8, 0xcb, 0xfb, 0xad, 0xc0, 0x5d, 0x1e, + 0x16, 0x39, 0x08, 0x0e, 0x42, 0x4a, 0x71, 0x40, 0x1e, 0xed, 0x82, 0xaa, 0x76, 0x14, 0x0b, 0x40, + 0x2d, 0xc2, 0x1c, 0x79, 0x1c, 0x18, 0x42, 0x50, 0x02, 0x83, 0xea, 0xbb, 0x99, 0x85, 0xe1, 0x4d, + 0x79, 0x7e, 0xf7, 0x27, 0x00, 0x00, 0xff, 0xff, 0x3f, 0x05, 0x0f, 0x90, 0x65, 0x05, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -131,7 +132,8 @@ type MasterClient interface { // // @return StringListResponse ShowPartitions(ctx context.Context, in *internalpb.ShowPartitionRequest, opts ...grpc.CallOption) (*servicepb.StringListResponse, error) - Tso(ctx context.Context, opts ...grpc.CallOption) (Master_TsoClient, error) + AllocTimestamp(ctx context.Context, in *internalpb.TsoRequest, opts ...grpc.CallOption) (*internalpb.TsoResponse, error) + AllocId(ctx context.Context, in *internalpb.IdRequest, opts ...grpc.CallOption) (*internalpb.IdResponse, error) } type masterClient struct { @@ -232,35 +234,22 @@ func (c *masterClient) ShowPartitions(ctx context.Context, in *internalpb.ShowPa return out, nil } -func (c *masterClient) Tso(ctx context.Context, opts ...grpc.CallOption) (Master_TsoClient, error) { - stream, err := c.cc.NewStream(ctx, &_Master_serviceDesc.Streams[0], "/milvus.proto.master.Master/Tso", opts...) +func (c *masterClient) AllocTimestamp(ctx context.Context, in *internalpb.TsoRequest, opts ...grpc.CallOption) (*internalpb.TsoResponse, error) { + out := new(internalpb.TsoResponse) + err := c.cc.Invoke(ctx, "/milvus.proto.master.Master/AllocTimestamp", in, out, opts...) if err != nil { return nil, err } - x := &masterTsoClient{stream} - return x, nil + return out, nil } -type Master_TsoClient interface { - Send(*internalpb.TsoRequest) error - Recv() (*internalpb.TsoResponse, error) - grpc.ClientStream -} - -type masterTsoClient struct { - grpc.ClientStream -} - -func (x *masterTsoClient) Send(m *internalpb.TsoRequest) error { - return x.ClientStream.SendMsg(m) -} - -func (x *masterTsoClient) Recv() (*internalpb.TsoResponse, error) { - m := new(internalpb.TsoResponse) - if err := x.ClientStream.RecvMsg(m); err != nil { +func (c *masterClient) AllocId(ctx context.Context, in *internalpb.IdRequest, opts ...grpc.CallOption) (*internalpb.IdResponse, error) { + out := new(internalpb.IdResponse) + err := c.cc.Invoke(ctx, "/milvus.proto.master.Master/AllocId", in, out, opts...) + if err != nil { return nil, err } - return m, nil + return out, nil } // MasterServer is the server API for Master service. @@ -325,7 +314,8 @@ type MasterServer interface { // // @return StringListResponse ShowPartitions(context.Context, *internalpb.ShowPartitionRequest) (*servicepb.StringListResponse, error) - Tso(Master_TsoServer) error + AllocTimestamp(context.Context, *internalpb.TsoRequest) (*internalpb.TsoResponse, error) + AllocId(context.Context, *internalpb.IdRequest) (*internalpb.IdResponse, error) } // UnimplementedMasterServer can be embedded to have forward compatible implementations. @@ -362,8 +352,11 @@ func (*UnimplementedMasterServer) DescribePartition(ctx context.Context, req *in func (*UnimplementedMasterServer) ShowPartitions(ctx context.Context, req *internalpb.ShowPartitionRequest) (*servicepb.StringListResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ShowPartitions not implemented") } -func (*UnimplementedMasterServer) Tso(srv Master_TsoServer) error { - return status.Errorf(codes.Unimplemented, "method Tso not implemented") +func (*UnimplementedMasterServer) AllocTimestamp(ctx context.Context, req *internalpb.TsoRequest) (*internalpb.TsoResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method AllocTimestamp not implemented") +} +func (*UnimplementedMasterServer) AllocId(ctx context.Context, req *internalpb.IdRequest) (*internalpb.IdResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method AllocId not implemented") } func RegisterMasterServer(s *grpc.Server, srv MasterServer) { @@ -550,30 +543,40 @@ func _Master_ShowPartitions_Handler(srv interface{}, ctx context.Context, dec fu return interceptor(ctx, in, info, handler) } -func _Master_Tso_Handler(srv interface{}, stream grpc.ServerStream) error { - return srv.(MasterServer).Tso(&masterTsoServer{stream}) -} - -type Master_TsoServer interface { - Send(*internalpb.TsoResponse) error - Recv() (*internalpb.TsoRequest, error) - grpc.ServerStream -} - -type masterTsoServer struct { - grpc.ServerStream -} - -func (x *masterTsoServer) Send(m *internalpb.TsoResponse) error { - return x.ServerStream.SendMsg(m) -} - -func (x *masterTsoServer) Recv() (*internalpb.TsoRequest, error) { - m := new(internalpb.TsoRequest) - if err := x.ServerStream.RecvMsg(m); err != nil { +func _Master_AllocTimestamp_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.TsoRequest) + if err := dec(in); err != nil { return nil, err } - return m, nil + if interceptor == nil { + return srv.(MasterServer).AllocTimestamp(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/milvus.proto.master.Master/AllocTimestamp", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(MasterServer).AllocTimestamp(ctx, req.(*internalpb.TsoRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Master_AllocId_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.IdRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(MasterServer).AllocId(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/milvus.proto.master.Master/AllocId", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(MasterServer).AllocId(ctx, req.(*internalpb.IdRequest)) + } + return interceptor(ctx, in, info, handler) } var _Master_serviceDesc = grpc.ServiceDesc{ @@ -620,14 +623,15 @@ var _Master_serviceDesc = grpc.ServiceDesc{ MethodName: "ShowPartitions", Handler: _Master_ShowPartitions_Handler, }, - }, - Streams: []grpc.StreamDesc{ { - StreamName: "Tso", - Handler: _Master_Tso_Handler, - ServerStreams: true, - ClientStreams: true, + MethodName: "AllocTimestamp", + Handler: _Master_AllocTimestamp_Handler, + }, + { + MethodName: "AllocId", + Handler: _Master_AllocId_Handler, }, }, + Streams: []grpc.StreamDesc{}, Metadata: "master.proto", } diff --git a/internal/proxy/manipulation_req.go b/internal/proxy/manipulation_req.go index d4a07bb10e..afc7f1e85c 100644 --- a/internal/proxy/manipulation_req.go +++ b/internal/proxy/manipulation_req.go @@ -2,14 +2,15 @@ package proxy import ( "fmt" + "log" + "sync" + "github.com/apache/pulsar-client-go/pulsar" "github.com/golang/protobuf/proto" "github.com/zilliztech/milvus-distributed/internal/errors" "github.com/zilliztech/milvus-distributed/internal/proto/commonpb" pb "github.com/zilliztech/milvus-distributed/internal/proto/message" "github.com/zilliztech/milvus-distributed/internal/util/typeutil" - "log" - "sync" ) type manipulationReq struct { @@ -65,8 +66,8 @@ func (req *manipulationReq) PostExecute() commonpb.Status { // send into pulsar func (req *manipulationReq) WaitToFinish() commonpb.Status { // wait until send into pulsar req.wg.Wait() - for _, stat := range req.stats{ - if stat.ErrorCode != commonpb.ErrorCode_SUCCESS{ + for _, stat := range req.stats { + if stat.ErrorCode != commonpb.ErrorCode_SUCCESS { return stat } } diff --git a/internal/proxy/manipulation_task.go b/internal/proxy/manipulation_task.go new file mode 100644 index 0000000000..f021b89b9e --- /dev/null +++ b/internal/proxy/manipulation_task.go @@ -0,0 +1,59 @@ +package proxy + +import ( + "github.com/zilliztech/milvus-distributed/internal/msgstream" + "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" + "github.com/zilliztech/milvus-distributed/internal/proto/servicepb" + "github.com/zilliztech/milvus-distributed/internal/util/typeutil" +) + +type insertTask struct { + baseTask + // SegIdAssigner, RowIdAllocator + rowBatch *servicepb.RowBatch + resultChan chan *servicepb.IntegerRangeResponse + pulsarMsgStream *msgstream.PulsarMsgStream +} + +func (it *insertTask) PreExecute() error { + return nil +} + +func (it *insertTask) Execute() error { + ts := it.GetTs() + insertRequest := internalpb.InsertRequest{ + ReqType: internalpb.ReqType_kInsert, + ReqId: it.ReqId, + CollectionName: it.rowBatch.CollectionName, + PartitionTag: it.rowBatch.PartitionTag, + SegmentId: 1, // TODO: use SegIdAssigner instead + // TODO: ChannelID + ProxyId: it.ProxyId, + Timestamps: []typeutil.Timestamp{ts}, + RowIds: []int64{1}, // TODO: use RowIdAllocator instead + RowData: it.rowBatch.RowData, + } + pulsarInsertTask := msgstream.InsertTask{ + InsertRequest: insertRequest, + } + var tsMsg msgstream.TsMsg = &pulsarInsertTask + msgPack := &msgstream.MsgPack{ + BeginTs: ts, + EndTs: ts, + } + msgPack.Msgs = append(msgPack.Msgs, &tsMsg) + it.pulsarMsgStream.Produce(msgPack) + return nil +} + +func (it *insertTask) PostExecute() error { + return nil +} + +func (it *insertTask) WaitToFinish() error { + return nil +} + +func (it *insertTask) Notify() error { + return nil +} diff --git a/internal/proxy/proxy_instance.go b/internal/proxy/proxy_instance.go new file mode 100644 index 0000000000..f1a788ac18 --- /dev/null +++ b/internal/proxy/proxy_instance.go @@ -0,0 +1,94 @@ +package proxy + +import ( + "context" + "log" + "net" + "sync" + + "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" + "github.com/zilliztech/milvus-distributed/internal/proto/servicepb" + "google.golang.org/grpc" +) + +type proxyInstance struct { + servicepb.UnimplementedMilvusServiceServer + grpcServer *grpc.Server + taskSch *taskScheduler + taskChan chan *task + ctx context.Context + cancel context.CancelFunc + wg sync.WaitGroup +} + +func (ins *proxyInstance) Insert(ctx context.Context, req *servicepb.RowBatch) (*servicepb.IntegerRangeResponse, error) { + return &servicepb.IntegerRangeResponse{}, nil +} + +func (ins *proxyInstance) StartGrpcServer() error { + // TODO: use address in config instead + lis, err := net.Listen("tcp", "127.0.0.1") + if err != nil { + return err + } + go func() { + ins.wg.Add(1) + defer ins.wg.Done() + server := grpc.NewServer() + servicepb.RegisterMilvusServiceServer(server, ins) + err := server.Serve(lis) + if err != nil { + log.Fatalf("Proxy grpc server fatal error=%v", err) + } + }() + return nil +} + +func (ins *proxyInstance) restartSchedulerRoutine(bufSize int) error { + ins.taskChan = make(chan *task, bufSize) + + go func() { + for { + select { + case t := <-ins.taskChan: + switch (*t).Type() { + case internalpb.ReqType_kInsert: + ins.taskSch.DmQueue.Enqueue(t) + default: + return + } + default: + return + } + } + }() + + return nil +} + +func (ins *proxyInstance) restartForwardRoutine() error { + + return nil +} + +func startProxyInstance(ins *proxyInstance) error { + if err := ins.restartSchedulerRoutine(1024); err != nil { + return err + } + if err := ins.restartForwardRoutine(); err != nil { + return err + } + + return ins.StartGrpcServer() +} + +func StartProxyInstance() error { + ins := &proxyInstance{} + err := startProxyInstance(ins) + if err != nil { + return nil + } + + ins.wg.Wait() + return nil +} diff --git a/internal/proxy/proxy_node.go b/internal/proxy/proxy_node.go index 7ee5914440..de7e2d04b0 100644 --- a/internal/proxy/proxy_node.go +++ b/internal/proxy/proxy_node.go @@ -3,6 +3,8 @@ package proxy import ( "context" "fmt" + "strconv" + "github.com/apache/pulsar-client-go/pulsar" "github.com/zilliztech/milvus-distributed/internal/allocator" "github.com/zilliztech/milvus-distributed/internal/conf" @@ -10,7 +12,6 @@ import ( "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" "github.com/zilliztech/milvus-distributed/internal/util/typeutil" etcd "go.etcd.io/etcd/clientv3" - "strconv" ) type BaseRequest interface { @@ -100,7 +101,7 @@ func StartProxy(opt *ProxyOptions) error { // saveInterval: opt.tsoSaveInterval, //} //tso.Restart(opt.proxyId) - tso := allocator.NewTimestampAllocator() + tso, _ := allocator.NewTimestampAllocator(opt.ctx) /////////////////// proxy server /////////////////////////////// //readerTopics, send insert and delete message into these topics diff --git a/internal/proxy/proxy_node_test.go b/internal/proxy/proxy_node_test.go index 3b7c110399..57fd673958 100644 --- a/internal/proxy/proxy_node_test.go +++ b/internal/proxy/proxy_node_test.go @@ -4,27 +4,35 @@ import ( "context" "encoding/binary" "encoding/json" - "github.com/apache/pulsar-client-go/pulsar" - "github.com/zilliztech/milvus-distributed/internal/proto/etcdpb" - "github.com/zilliztech/milvus-distributed/internal/allocator" - pb "github.com/zilliztech/milvus-distributed/internal/proto/message" - "github.com/golang/protobuf/proto" - "github.com/stretchr/testify/assert" - "github.com/zilliztech/milvus-distributed/internal/util/tsoutil" - etcd "go.etcd.io/etcd/clientv3" - "google.golang.org/grpc" + "os" "sort" "strconv" "testing" "time" -) + "github.com/apache/pulsar-client-go/pulsar" + "github.com/golang/protobuf/proto" + "github.com/stretchr/testify/assert" + "github.com/zilliztech/milvus-distributed/internal/allocator" + "github.com/zilliztech/milvus-distributed/internal/proto/etcdpb" + pb "github.com/zilliztech/milvus-distributed/internal/proto/message" + "github.com/zilliztech/milvus-distributed/internal/util/tsoutil" + etcd "go.etcd.io/etcd/clientv3" + "google.golang.org/grpc" +) const ( tsoKeyPath string = "/timestampOracle" ) -var timeAllocator *allocator.TimestampAllocator = allocator.NewTimestampAllocator() +var timeAllocator *allocator.TimestampAllocator + +func TestMain(m *testing.M) { + timeAllocator, _ = allocator.NewTimestampAllocator(context.Background()) + exitCode := m.Run() + timeAllocator.Close() + os.Exit(exitCode) +} func TestProxyNode(t *testing.T) { startTestMaster("localhost:11000", t) @@ -75,10 +83,10 @@ func TestProxyNode(t *testing.T) { assert.Nil(t, err) curValue, err := testOpt.tso.AllocOne() - curTS, err := timeAllocator.AllocOne() + curTS, err := timeAllocator.AllocOne() assert.Equalf(t, err, nil, "%s", "allocator failed") - curTime, _:= tsoutil.ParseTS(curTS) + curTime, _ := tsoutil.ParseTS(curTS) t.Logf("current time stamp = %d, saved time stamp = %d", curTime, value) assert.GreaterOrEqual(t, curValue, value) assert.GreaterOrEqual(t, value, startTime) @@ -163,7 +171,7 @@ func TestProxyNode(t *testing.T) { if err := proto.Unmarshal(cm.Payload(), &tsm); err != nil { t.Fatal(err) } - curT, _:= tsoutil.ParseTS(tsm.Timestamp) + curT, _ := tsoutil.ParseTS(tsm.Timestamp) t.Logf("time tick = %d", curT) assert.Greater(t, curT, lastT) lastT = curT @@ -252,7 +260,7 @@ func TestProxyNode(t *testing.T) { physicalTime, _ := tsoutil.ParseTS(qm.Timestamp) t.Logf("query time stamp = %d", physicalTime) - assert.Greater(t,physicalTime, startTime) + assert.Greater(t, physicalTime, startTime) r1 := pb.QueryResult{ Status: &pb.Status{ErrorCode: pb.ErrorCode_SUCCESS}, diff --git a/internal/proxy/query_req.go b/internal/proxy/query_req.go index b47c2e54fb..d158f0f239 100644 --- a/internal/proxy/query_req.go +++ b/internal/proxy/query_req.go @@ -1,13 +1,14 @@ package proxy import ( + "log" + "sync" + "github.com/apache/pulsar-client-go/pulsar" "github.com/golang/protobuf/proto" "github.com/zilliztech/milvus-distributed/internal/proto/commonpb" "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" "github.com/zilliztech/milvus-distributed/internal/proto/servicepb" - "log" - "sync" ) type queryReq struct { @@ -80,7 +81,6 @@ func (s *proxyServer) restartQueryRoutine(buf_size int) error { } qm.Timestamp = uint64(ts[0]) - qb, err := proto.Marshal(qm) if err != nil { log.Printf("Marshal QueryReqMsg failed, error = %v", err) @@ -141,22 +141,22 @@ func (s *proxyServer) reduceResults(query *queryReq) *servicepb.QueryResult { status = *r.Status if status.ErrorCode == commonpb.ErrorCode_SUCCESS { results = append(results, r) - }else{ + } else { break } } - if len(results) != s.numReaderNode{ + if len(results) != s.numReaderNode { status.ErrorCode = commonpb.ErrorCode_UNEXPECTED_ERROR } - if status.ErrorCode != commonpb.ErrorCode_SUCCESS{ - result:= servicepb.QueryResult{ + if status.ErrorCode != commonpb.ErrorCode_SUCCESS { + result := servicepb.QueryResult{ Status: &status, } return &result } if s.numReaderNode == 1 { - result:= servicepb.QueryResult{ + result := servicepb.QueryResult{ Status: &commonpb.Status{ ErrorCode: commonpb.ErrorCode_SUCCESS, }, diff --git a/internal/proxy/readertimesync.go b/internal/proxy/readertimesync.go deleted file mode 100644 index a049fccb5f..0000000000 --- a/internal/proxy/readertimesync.go +++ /dev/null @@ -1,372 +0,0 @@ -package proxy - -import ( - "context" - "fmt" - "github.com/apache/pulsar-client-go/pulsar" - "github.com/zilliztech/milvus-distributed/internal/conf" - pb "github.com/zilliztech/milvus-distributed/internal/proto/message" - "github.com/golang/protobuf/proto" - "log" - "sort" - "strconv" - "sync" -) - -const ReadStopFlagEnd int64 = 0 - -type ReaderTimeSync interface { - Start() error - Close() - TimeSync() <-chan TimeSyncMsg - ManipulationReqMsg() <-chan *pb.ManipulationReqMsg - IsManipulationReqMsgChanFull() bool -} - -type TimeSyncMsg struct { - Timestamp uint64 - NumRecorders int64 -} - -type ReaderTimeSyncOption func(*ReaderTimeSyncCfg) - -type ReaderTimeSyncCfg struct { - pulsarAddr string - pulsarClient pulsar.Client - - timeSyncConsumer pulsar.Consumer - readerConsumer pulsar.Consumer - readerProducer []pulsar.Producer - - timesyncMsgChan chan TimeSyncMsg - manipulationReqMsgChan chan *pb.ManipulationReqMsg //output insert or delete msg - - readStopFlagClientId int64 - interval int64 - proxyIdList []int64 - readerQueueSize int - - revTimesyncFromReader map[uint64]int - - ctx context.Context - cancel context.CancelFunc -} - -/* -layout of timestamp - time ms logic number -/-------46 bit-----------\/------18bit-----\ -+-------------------------+================+ -*/ -func toMillisecond(ts *pb.TimeSyncMsg) int { - // get Millisecond in second - return int(ts.GetTimestamp() >> 18) -} - -func NewReaderTimeSync( - timeSyncTopic string, - timeSyncSubName string, - readTopics []string, - readSubName string, - proxyIdList []int64, - readStopFlagClientId int64, - opts ...ReaderTimeSyncOption, -) (ReaderTimeSync, error) { - //pulsarAddr := "pulsar://" - //pulsarAddr += conf.Config.Pulsar.Address - //pulsarAddr += ":" - //pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10) - //interval := int64(conf.Config.Timesync.Interval) - - //check if proxyId has duplication - if len(proxyIdList) == 0 { - return nil, fmt.Errorf("proxy id list is empty") - } - if len(proxyIdList) > 1 { - sort.Slice(proxyIdList, func(i int, j int) bool { return proxyIdList[i] < proxyIdList[j] }) - } - for i := 1; i < len(proxyIdList); i++ { - if proxyIdList[i] == proxyIdList[i-1] { - return nil, fmt.Errorf("there are two proxies have the same id = %d", proxyIdList[i]) - } - } - r := &ReaderTimeSyncCfg{ - //interval: interval, - proxyIdList: proxyIdList, - } - for _, opt := range opts { - opt(r) - } - if r.interval == 0 { - r.interval = int64(conf.Config.Timesync.Interval) - if r.interval == 0 { - return nil, fmt.Errorf("interval is unsetted") - } - } - if len(r.pulsarAddr) == 0 { - pulsarAddr := "pulsar://" - pulsarAddr += conf.Config.Pulsar.Address - pulsarAddr += ":" - pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10) - r.pulsarAddr = pulsarAddr - } - - //check if read topic is empty - if len(readTopics) == 0 { - return nil, fmt.Errorf("read topic is empyt") - } - //set default value - if r.readerQueueSize == 0 { - r.readerQueueSize = 1024 - } - if readStopFlagClientId >= ReadStopFlagEnd { - return nil, fmt.Errorf("read stop flag client id should less than %d", ReadStopFlagEnd) - } - r.readStopFlagClientId = readStopFlagClientId - - r.timesyncMsgChan = make(chan TimeSyncMsg, len(readTopics)*r.readerQueueSize) - r.manipulationReqMsgChan = make(chan *pb.ManipulationReqMsg, len(readTopics)*r.readerQueueSize) - r.revTimesyncFromReader = make(map[uint64]int) - r.ctx, r.cancel = context.WithCancel(context.Background()) - - client, err := pulsar.NewClient(pulsar.ClientOptions{URL: r.pulsarAddr}) - if err != nil { - return nil, fmt.Errorf("connect pulsar failed, %v", err) - } - r.pulsarClient = client - - timeSyncChan := make(chan pulsar.ConsumerMessage, len(r.proxyIdList)) - if r.timeSyncConsumer, err = r.pulsarClient.Subscribe(pulsar.ConsumerOptions{ - Topic: timeSyncTopic, - SubscriptionName: timeSyncSubName, - Type: pulsar.KeyShared, - SubscriptionInitialPosition: pulsar.SubscriptionPositionEarliest, - MessageChannel: timeSyncChan, - }); err != nil { - return nil, fmt.Errorf("failed to subscribe topic %s, error = %v", timeSyncTopic, err) - } - - readerChan := make(chan pulsar.ConsumerMessage, len(readTopics)*r.readerQueueSize) - if r.readerConsumer, err = r.pulsarClient.Subscribe(pulsar.ConsumerOptions{ - Topics: readTopics, - SubscriptionName: readSubName, - Type: pulsar.KeyShared, - SubscriptionInitialPosition: pulsar.SubscriptionPositionEarliest, - MessageChannel: readerChan, - }); err != nil { - return nil, fmt.Errorf("failed to subscrive reader topics : %v, error = %v", readTopics, err) - } - - r.readerProducer = make([]pulsar.Producer, 0, len(readTopics)) - for i := 0; i < len(readTopics); i++ { - rp, err := r.pulsarClient.CreateProducer(pulsar.ProducerOptions{Topic: readTopics[i]}) - if err != nil { - return nil, fmt.Errorf("failed to create reader producer %s, error = %v", readTopics[i], err) - } - r.readerProducer = append(r.readerProducer, rp) - } - - return r, nil -} - -func (r *ReaderTimeSyncCfg) Close() { - r.cancel() - r.timeSyncConsumer.Close() - r.readerConsumer.Close() - for i := 0; i < len(r.readerProducer); i++ { - r.readerProducer[i].Close() - } - r.pulsarClient.Close() -} - -func (r *ReaderTimeSyncCfg) Start() error { - go r.startReadTopics() - go r.startTimeSync() - return r.ctx.Err() -} - -func (r *ReaderTimeSyncCfg) ManipulationReqMsg() <-chan *pb.ManipulationReqMsg { - return r.manipulationReqMsgChan -} - -func (r *ReaderTimeSyncCfg) TimeSync() <-chan TimeSyncMsg { - return r.timesyncMsgChan -} - -func (r *ReaderTimeSyncCfg) TimeSyncChanLen() int { - return len(r.timesyncMsgChan) -} - -func (r *ReaderTimeSyncCfg) IsManipulationReqMsgChanFull() bool { - return len(r.manipulationReqMsgChan) == len(r.readerProducer)*r.readerQueueSize -} - -func (r *ReaderTimeSyncCfg) alignTimeSync(ts []*pb.TimeSyncMsg) []*pb.TimeSyncMsg { - if len(r.proxyIdList) > 1 { - if len(ts) > 1 { - for i := 1; i < len(r.proxyIdList); i++ { - curIdx := len(ts) - 1 - i - preIdx := len(ts) - i - timeGap := toMillisecond(ts[curIdx]) - toMillisecond(ts[preIdx]) - if int64(timeGap) >= (r.interval/2) || int64(timeGap) <= (-r.interval/2) { - ts = ts[preIdx:] - return ts - } - } - ts = ts[len(ts)-len(r.proxyIdList):] - sort.Slice(ts, func(i int, j int) bool { return ts[i].Peer_Id < ts[j].Peer_Id }) - for i := 0; i < len(r.proxyIdList); i++ { - if ts[i].Peer_Id != r.proxyIdList[i] { - ts = ts[:0] - return ts - } - } - } - } else { - if len(ts) > 1 { - ts = ts[len(ts)-1:] - } - } - return ts -} - -func (r *ReaderTimeSyncCfg) readTimeSync(ctx context.Context, ts []*pb.TimeSyncMsg, n int) ([]*pb.TimeSyncMsg, error) { - for i := 0; i < n; i++ { - select { - case <-ctx.Done(): - return nil, ctx.Err() - case cm, ok := <-r.timeSyncConsumer.Chan(): - if ok == false { - return nil, fmt.Errorf("timesync consumer closed") - } - - msg := cm.Message - var tsm pb.TimeSyncMsg - if err := proto.Unmarshal(msg.Payload(), &tsm); err != nil { - return nil, err - } - - ts = append(ts, &tsm) - r.timeSyncConsumer.AckID(msg.ID()) - } - } - return ts, nil -} - -func (r *ReaderTimeSyncCfg) sendEOFMsg(ctx context.Context, msg *pulsar.ProducerMessage, index int, wg *sync.WaitGroup) { - if _, err := r.readerProducer[index].Send(ctx, msg); err != nil { - //TODO, log error - log.Printf("Send timesync flag error %v", err) - } - wg.Done() -} - -func (r *ReaderTimeSyncCfg) startTimeSync() { - tsm := make([]*pb.TimeSyncMsg, 0, len(r.proxyIdList)*2) - ctx, _ := context.WithCancel(r.ctx) - var err error - for { - //var start time.Time - for len(tsm) != len(r.proxyIdList) { - tsm = r.alignTimeSync(tsm) - tsm, err = r.readTimeSync(ctx, tsm, len(r.proxyIdList)-len(tsm)) - if err != nil { - if ctx.Err() != nil { - return - } else { - //TODO, log error msg - log.Printf("read time sync error %v", err) - } - } - } - ts := tsm[0].Timestamp - for i := 1; i < len(tsm); i++ { - if tsm[i].Timestamp < ts { - ts = tsm[i].Timestamp - } - } - tsm = tsm[:0] - //send timestamp flag to reader channel - msg := pb.ManipulationReqMsg{Timestamp: ts, ProxyId: r.readStopFlagClientId} - payload, err := proto.Marshal(&msg) - if err != nil { - //TODO log error - log.Printf("Marshal timesync flag error %v", err) - } else { - wg := sync.WaitGroup{} - wg.Add(len(r.readerProducer)) - for index := range r.readerProducer { - go r.sendEOFMsg(ctx, &pulsar.ProducerMessage{Payload: payload}, index, &wg) - } - wg.Wait() - } - } -} - -func (r *ReaderTimeSyncCfg) isReadStopFlag(imsg *pb.ManipulationReqMsg) bool { - return imsg.ProxyId < ReadStopFlagEnd -} - -func (r *ReaderTimeSyncCfg) startReadTopics() { - ctx, _ := context.WithCancel(r.ctx) - tsm := TimeSyncMsg{Timestamp: 0, NumRecorders: 0} - for { - select { - case <-ctx.Done(): - return - case cm, ok := <-r.readerConsumer.Chan(): - if ok == false { - //TODO,log error - log.Printf("reader consumer closed") - } - msg := cm.Message - var imsg pb.ManipulationReqMsg - if err := proto.Unmarshal(msg.Payload(), &imsg); err != nil { - //TODO, log error - log.Printf("unmarshal InsertOrDeleteMsg error %v", err) - break - } - if r.isReadStopFlag(&imsg) { //timestamp flag - if imsg.ProxyId == r.readStopFlagClientId { - gval := r.revTimesyncFromReader[imsg.Timestamp] - gval++ - if gval >= len(r.readerProducer) { - if imsg.Timestamp >= tsm.Timestamp { - tsm.Timestamp = imsg.Timestamp - r.timesyncMsgChan <- tsm - tsm.NumRecorders = 0 - } - delete(r.revTimesyncFromReader, imsg.Timestamp) - } else { - r.revTimesyncFromReader[imsg.Timestamp] = gval - } - } - } else { - if r.IsManipulationReqMsgChanFull() { - log.Printf("WARN : Insert or delete chan is full ...") - } - tsm.NumRecorders++ - r.manipulationReqMsgChan <- &imsg - } - r.readerConsumer.AckID(msg.ID()) - } - } -} - -func WithReaderQueueSize(size int) ReaderTimeSyncOption { - return func(r *ReaderTimeSyncCfg) { - r.readerQueueSize = size - } -} - -func WithPulsarAddress(addr string) ReaderTimeSyncOption { - return func(r *ReaderTimeSyncCfg) { - r.pulsarAddr = addr - } -} - -func WithInterval(interval int64) ReaderTimeSyncOption { - return func(r *ReaderTimeSyncCfg) { - r.interval = interval - } -} diff --git a/internal/proxy/readertimesync_test.go b/internal/proxy/readertimesync_test.go deleted file mode 100644 index dc386e135e..0000000000 --- a/internal/proxy/readertimesync_test.go +++ /dev/null @@ -1,564 +0,0 @@ -package proxy - -import ( - "context" - "github.com/apache/pulsar-client-go/pulsar" - pb "github.com/zilliztech/milvus-distributed/internal/proto/message" - "github.com/golang/protobuf/proto" - "github.com/stretchr/testify/assert" - "log" - "sync" - "testing" - "time" -) - -const ( - timeSyncTopic = "rtimesync" - timeSyncTopic2 = "rtimesync2" - timeSyncTopic3 = "rtimesync3" - timeSyncSubName = "rtimesync-g" - timeSyncSubName1 = "rtimesync-g1" - timeSyncSubName2 = "rtimesync-g2" - timeSyncSubName3 = "rtimesync-g3" - readerTopic1 = "rreader1" - readerTopic12 = "rreader12" - readerTopic13 = "rreader13" - readerTopic2 = "rreader2" - readerTopic22 = "rreader22" - readerTopic23 = "rreader23" - readerTopic3 = "rreader3" - readerTopic32 = "rreader32" - readerTopic33 = "rreader33" - readerTopic4 = "rreader4" - readerTopic42 = "rreader42" - readerTopic43 = "rreader43" - readerSubName = "rreader-g" - readerSubName1 = "rreader-g1" - readerSubName2 = "rreader-g2" - readerSubName3 = "rreader-g3" - interval = 200 - readStopFlag int64 = -1 - readStopFlag1 int64 = -1 - readStopFlag2 int64 = -2 - readStopFlag3 int64 = -3 -) - -func TestAlignTimeSync(t *testing.T) { - r := &ReaderTimeSyncCfg{ - proxyIdList: []int64{1, 2, 3}, - interval: 200, - } - ts := []*pb.TimeSyncMsg{ - { - Peer_Id: 1, - Timestamp: toTimestamp(5), - }, - { - Peer_Id: 3, - Timestamp: toTimestamp(15), - }, - { - Peer_Id: 2, - Timestamp: toTimestamp(20), - }, - } - r.alignTimeSync(ts) - assert.Equalf(t, len(r.proxyIdList), 3, "proxyIdList should be : 1 2 3") - for i := 0; i < len(r.proxyIdList); i++ { - assert.Equal(t, r.proxyIdList[i], ts[i].Peer_Id) - } - -} - -func TestAlignTimeSync2(t *testing.T) { - r := &ReaderTimeSyncCfg{ - proxyIdList: []int64{1, 2, 3}, - interval: 200, - } - ts := []*pb.TimeSyncMsg{ - { - Peer_Id: 1, - Timestamp: toTimestamp(5), - }, - { - Peer_Id: 3, - Timestamp: toTimestamp(150), - }, - { - Peer_Id: 2, - Timestamp: toTimestamp(20), - }, - } - ts = r.alignTimeSync(ts) - assert.Equalf(t, len(r.proxyIdList), 3, "proxyIdList should be : 1 2 3") - assert.Equal(t, len(ts), 1) - assert.Equal(t, ts[0].Peer_Id, int64(2)) -} - -func TestAlignTimeSync3(t *testing.T) { - r := &ReaderTimeSyncCfg{ - proxyIdList: []int64{1, 2, 3}, - interval: 200, - } - ts := []*pb.TimeSyncMsg{ - { - Peer_Id: 1, - Timestamp: toTimestamp(5), - }, - { - Peer_Id: 1, - Timestamp: toTimestamp(5), - }, - { - Peer_Id: 1, - Timestamp: toTimestamp(5), - }, - { - Peer_Id: 3, - Timestamp: toTimestamp(15), - }, - { - Peer_Id: 2, - Timestamp: toTimestamp(20), - }, - } - ts = r.alignTimeSync(ts) - assert.Equalf(t, len(r.proxyIdList), 3, "proxyIdList should be : 1 2 3") - for i := 0; i < len(r.proxyIdList); i++ { - assert.Equal(t, r.proxyIdList[i], ts[i].Peer_Id) - } -} - -func TestAlignTimeSync4(t *testing.T) { - r := &ReaderTimeSyncCfg{ - proxyIdList: []int64{1}, - interval: 200, - } - ts := []*pb.TimeSyncMsg{ - { - Peer_Id: 1, - Timestamp: toTimestamp(15), - }, - { - Peer_Id: 1, - Timestamp: toTimestamp(25), - }, - { - Peer_Id: 1, - Timestamp: toTimestamp(35), - }, - } - ts = r.alignTimeSync(ts) - assert.Equalf(t, len(r.proxyIdList), 1, "proxyIdList should be : 1") - assert.Equal(t, len(ts), 1) - assert.Equal(t, getMillisecond(ts[0].Timestamp), uint64(35)) -} - -func TestAlignTimeSync5(t *testing.T) { - r := &ReaderTimeSyncCfg{ - proxyIdList: []int64{1, 2, 3}, - interval: 200, - } - ts := []*pb.TimeSyncMsg{ - { - Peer_Id: 1, - Timestamp: toTimestamp(5), - }, - { - Peer_Id: 1, - Timestamp: toTimestamp(5), - }, - { - Peer_Id: 1, - Timestamp: toTimestamp(5), - }, - { - Peer_Id: 3, - Timestamp: toTimestamp(15), - }, - { - Peer_Id: 3, - Timestamp: toTimestamp(20), - }, - } - ts = r.alignTimeSync(ts) - assert.Zero(t, len(ts)) -} - -func TestNewReaderTimeSync(t *testing.T) { - r, err := NewReaderTimeSync( - timeSyncTopic, - timeSyncSubName, - []string{readerTopic1, readerTopic2, readerTopic3, readerTopic4}, - readerSubName, - []int64{2, 1}, - readStopFlag, - WithPulsarAddress("pulsar://localhost:6650"), - WithInterval(interval), - WithReaderQueueSize(8), - ) - assert.Nil(t, err) - - rr := r.(*ReaderTimeSyncCfg) - assert.NotNil(t, rr.pulsarClient) - assert.NotNil(t, rr.timeSyncConsumer) - assert.NotNil(t, rr.readerConsumer) - assert.NotNil(t, rr.readerProducer) - assert.Equal(t, rr.interval, int64(interval)) - assert.Equal(t, rr.readStopFlagClientId, int64(readStopFlag)) - assert.Equal(t, rr.readerQueueSize, 8) - assert.Equal(t, len(rr.proxyIdList), 2) - assert.Equal(t, rr.proxyIdList[0], int64(1)) - assert.Equal(t, rr.proxyIdList[1], int64(2)) - r.Close() -} - -func TestPulsarClient(t *testing.T) { - t.Skip("skip pulsar client") - client, err := pulsar.NewClient(pulsar.ClientOptions{URL: "pulsar://localhost:6650"}) - assert.Nil(t, err) - - ctx, _ := context.WithTimeout(context.Background(), 3*time.Second) - go startWriteTimeSync(1, timeSyncTopic, client, 2*time.Second, t) - go startWriteTimeSync(2, timeSyncTopic, client, 2*time.Second, t) - timeSyncChan := make(chan pulsar.ConsumerMessage) - consumer, err := client.Subscribe(pulsar.ConsumerOptions{ - Topic: timeSyncTopic, - SubscriptionName: timeSyncSubName, - Type: pulsar.KeyShared, - SubscriptionInitialPosition: pulsar.SubscriptionPositionEarliest, - MessageChannel: timeSyncChan, - }) - assert.Nil(t, err) - - for { - select { - case cm := <-timeSyncChan: - msg := cm.Message - var tsm pb.TimeSyncMsg - if err := proto.Unmarshal(msg.Payload(), &tsm); err != nil { - log.Fatal(err) - } - consumer.AckID(msg.ID()) - log.Printf("read time stamp, id = %d, time stamp = %d\n", tsm.Peer_Id, tsm.Timestamp) - case <-ctx.Done(): - break - } - if ctx.Err() != nil { - break - } - } -} - -func TestReaderTimesync(t *testing.T) { - r, err := NewReaderTimeSync(timeSyncTopic, - timeSyncSubName, - []string{readerTopic1, readerTopic2, readerTopic3, readerTopic4}, - readerSubName, - []int64{2, 1}, - readStopFlag, - WithPulsarAddress("pulsar://localhost:6650"), - WithInterval(interval), - WithReaderQueueSize(1024), - ) - assert.Nil(t, err) - - rr := r.(*ReaderTimeSyncCfg) - pt1, err := rr.pulsarClient.CreateProducer(pulsar.ProducerOptions{Topic: timeSyncTopic}) - assert.Nil(t, err) - - pt2, err := rr.pulsarClient.CreateProducer(pulsar.ProducerOptions{Topic: timeSyncTopic}) - assert.Nil(t, err) - - pr1, err := rr.pulsarClient.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic1}) - assert.Nil(t, err) - - pr2, err := rr.pulsarClient.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic2}) - assert.Nil(t, err) - - pr3, err := rr.pulsarClient.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic3}) - assert.Nil(t, err) - - pr4, err := rr.pulsarClient.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic4}) - assert.Nil(t, err) - - go startProxy(pt1, 1, pr1, 1, pr2, 2, 2*time.Second, t) - go startProxy(pt2, 2, pr3, 3, pr4, 4, 2*time.Second, t) - - ctx, _ := context.WithTimeout(context.Background(), 3*time.Second) - r.Start() - - var tsm1, tsm2 TimeSyncMsg - var totalRecordes int64 = 0 - for { - if ctx.Err() != nil { - break - } - select { - case <-ctx.Done(): - tsm1.NumRecorders = 0 - break - case tsm1 = <-r.TimeSync(): - - } - if tsm1.NumRecorders > 0 { - log.Printf("timestamp %d, num records = %d", getMillisecond(tsm1.Timestamp), tsm1.NumRecorders) - totalRecordes += tsm1.NumRecorders - for i := int64(0); i < tsm1.NumRecorders; i++ { - im := <-r.ManipulationReqMsg() - //log.Printf("%d - %d", getMillisecond(im.Timestamp), getMillisecond(tsm2.Timestamp)) - if im.Timestamp < tsm2.Timestamp { - t.Fatalf("time sync error , im.Timestamp = %d, tsm2.Timestamp = %d", im.Timestamp, tsm2.Timestamp) - } - } - tsm2 = tsm1 - } - - } - log.Printf("total recordes = %d", totalRecordes) - if totalRecordes != 800 { - t.Fatalf("total records should be 800") - } - r.Close() - pt1.Close() - pt2.Close() - pr1.Close() - pr2.Close() - pr3.Close() - pr4.Close() -} - -func TestReaderTimesync2(t *testing.T) { - client, _ := pulsar.NewClient(pulsar.ClientOptions{URL: "pulsar://localhost:6650"}) - pt1, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: timeSyncTopic2}) - pt2, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: timeSyncTopic2}) - pr1, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic12}) - pr2, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic22}) - pr3, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic32}) - pr4, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic42}) - - go startProxy(pt1, 1, pr1, 1, pr2, 2, 2*time.Second, t) - go startProxy(pt2, 2, pr3, 3, pr4, 4, 2*time.Second, t) - - r1, _ := NewReaderTimeSync(timeSyncTopic2, - timeSyncSubName1, - []string{readerTopic12, readerTopic22, readerTopic32, readerTopic42}, - readerSubName1, - []int64{2, 1}, - readStopFlag1, - WithPulsarAddress("pulsar://localhost:6650"), - WithInterval(interval), - WithReaderQueueSize(1024), - ) - - r2, _ := NewReaderTimeSync(timeSyncTopic2, - timeSyncSubName2, - []string{readerTopic12, readerTopic22, readerTopic32, readerTopic42}, - readerSubName2, - []int64{2, 1}, - readStopFlag2, - WithPulsarAddress("pulsar://localhost:6650"), - WithInterval(interval), - WithReaderQueueSize(1024), - ) - - ctx, _ := context.WithTimeout(context.Background(), 3*time.Second) - rt := []ReaderTimeSync{r1, r2} - var wg sync.WaitGroup - for _, r := range rt { - r := r - _ = r.Start() - wg.Add(1) - go func() { - var tsm1, tsm2 TimeSyncMsg - var totalRecordes int64 = 0 - work := false - defer wg.Done() - for { - if ctx.Err() != nil { - break - } - select { - case tsm1 = <-r.TimeSync(): - work = true - default: - work = false - } - if work { - if tsm1.NumRecorders > 0 { - //log.Printf("timestamp %d, num records = %d", getMillisecond(tsm1.Timestamp), tsm1.NumRecorders) - totalRecordes += tsm1.NumRecorders - for i := int64(0); i < tsm1.NumRecorders; i++ { - im := <-r.ManipulationReqMsg() - //log.Printf("%d - %d", getMillisecond(im.Timestamp), getMillisecond(tsm2.Timestamp)) - assert.GreaterOrEqual(t, im.Timestamp, tsm2.Timestamp) - } - tsm2 = tsm1 - } - } - } - log.Printf("total recordes = %d", totalRecordes) - assert.Equal(t, totalRecordes, int64(800)) - }() - } - wg.Wait() - r1.Close() - r2.Close() - pt1.Close() - pt2.Close() - pr1.Close() - pr2.Close() - pr3.Close() - pr4.Close() -} - -func TestReaderTimesync3(t *testing.T) { - client, _ := pulsar.NewClient(pulsar.ClientOptions{URL: "pulsar://localhost:6650"}) - pt, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: timeSyncTopic3}) - pr1, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic13}) - pr2, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic23}) - pr3, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic33}) - pr4, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: readerTopic43}) - defer func() { - pr1.Close() - pr2.Close() - pr3.Close() - pr4.Close() - pt.Close() - client.Close() - }() - go func() { - total := 2 * 1000 / 10 - ticker := time.Tick(10 * time.Millisecond) - var timestamp uint64 = 0 - prlist := []pulsar.Producer{pr1, pr2, pr3, pr4} - for i := 1; i <= total; i++ { - <-ticker - timestamp += 10 - for idx, pr := range prlist { - msg := pb.ManipulationReqMsg{ProxyId: int64(idx + 1), Timestamp: toTimestamp(timestamp)} - mb, err := proto.Marshal(&msg) - assert.Nil(t, err) - if _, err := pr.Send(context.Background(), &pulsar.ProducerMessage{Payload: mb}); err != nil { - t.Fatal(err) - } - } - if i%20 == 0 { - tm := pb.TimeSyncMsg{Peer_Id: 1, Timestamp: toTimestamp(timestamp)} - tb, err := proto.Marshal(&tm) - assert.Nil(t, err) - if _, err := pt.Send(context.Background(), &pulsar.ProducerMessage{Payload: tb}); err != nil { - t.Fatal(err) - } - } - } - }() - - r, err := NewReaderTimeSync(timeSyncTopic3, - timeSyncSubName3, - []string{readerTopic13, readerTopic23, readerTopic33, readerTopic43}, - readerSubName3, - []int64{1}, - readStopFlag3, - WithPulsarAddress("pulsar://localhost:6650"), - WithInterval(interval), - WithReaderQueueSize(1024)) - assert.Nil(t, err) - defer r.Close() - - ctx, _ := context.WithTimeout(context.Background(), 3*time.Second) - if err := r.Start(); err != nil { - t.Fatal(err) - } - var tsm1, tsm2 TimeSyncMsg - var totalRecords int64 = 0 - for { - if ctx.Err() != nil { - break - } - select { - case <-ctx.Done(): - tsm1.NumRecorders = 0 - break - case tsm1 = <-r.TimeSync(): - - } - if tsm1.NumRecorders > 0 { - totalRecords += tsm1.NumRecorders - for i := int64(0); i < tsm1.NumRecorders; i++ { - im := <-r.ManipulationReqMsg() - assert.GreaterOrEqual(t, im.Timestamp, tsm2.Timestamp) - } - tsm2 = tsm1 - } - } - log.Printf("total records = %d", totalRecords) - assert.Equal(t, totalRecords, int64(800)) -} - -func getMillisecond(ts uint64) uint64 { - return ts >> 18 -} - -func toTimestamp(ts uint64) uint64 { - return ts << 18 -} - -func startWriteTimeSync(id int64, topic string, client pulsar.Client, duration time.Duration, t *testing.T) { - p, _ := client.CreateProducer(pulsar.ProducerOptions{Topic: topic}) - ticker := time.Tick(interval * time.Millisecond) - numSteps := int(duration / (interval * time.Millisecond)) - var tm uint64 = 0 - for i := 0; i < numSteps; i++ { - <-ticker - tm += interval - tsm := pb.TimeSyncMsg{Timestamp: toTimestamp(tm), Peer_Id: id} - tb, _ := proto.Marshal(&tsm) - if _, err := p.Send(context.Background(), &pulsar.ProducerMessage{Payload: tb}); err != nil { - t.Fatalf("send failed tsm id=%d, timestamp=%d, err=%v", tsm.Peer_Id, tsm.Timestamp, err) - } else { - //log.Printf("send tsm id=%d, timestamp=%d", tsm.Peer_Id, tsm.Timestamp) - } - } -} - -func startProxy(pt pulsar.Producer, ptid int64, pr1 pulsar.Producer, prid1 int64, pr2 pulsar.Producer, prid2 int64, duration time.Duration, t *testing.T) { - total := int(duration / (10 * time.Millisecond)) - ticker := time.Tick(10 * time.Millisecond) - var timestamp uint64 = 0 - for i := 1; i <= total; i++ { - <-ticker - timestamp += 10 - msg := pb.ManipulationReqMsg{ProxyId: int64(prid1), Timestamp: toTimestamp(timestamp)} - mb, err := proto.Marshal(&msg) - if err != nil { - t.Fatalf("marshal error %v", err) - } - if _, err := pr1.Send(context.Background(), &pulsar.ProducerMessage{Payload: mb}); err != nil { - t.Fatalf("send msg error %v", err) - } - - msg.ProxyId = prid2 - mb, err = proto.Marshal(&msg) - if err != nil { - t.Fatalf("marshal error %v", err) - } - if _, err := pr2.Send(context.Background(), &pulsar.ProducerMessage{Payload: mb}); err != nil { - t.Fatalf("send msg error %v", err) - } - - //log.Printf("send msg id = [ %d %d ], timestamp = %d", prid1, prid2, timestamp) - - if i%20 == 0 { - tm := pb.TimeSyncMsg{Peer_Id: ptid, Timestamp: toTimestamp(timestamp)} - tb, err := proto.Marshal(&tm) - if err != nil { - t.Fatalf("marshal error %v", err) - } - if _, err := pt.Send(context.Background(), &pulsar.ProducerMessage{Payload: tb}); err != nil { - t.Fatalf("send msg error %v", err) - } - //log.Printf("send timestamp id = %d, timestamp = %d", ptid, timestamp) - } - } -} diff --git a/internal/proxy/request_scheduler.go b/internal/proxy/request_scheduler.go index e23d2920fb..71aa72a252 100644 --- a/internal/proxy/request_scheduler.go +++ b/internal/proxy/request_scheduler.go @@ -1,8 +1,9 @@ package proxy import ( - "github.com/zilliztech/milvus-distributed/internal/util/typeutil" "sync" + + "github.com/zilliztech/milvus-distributed/internal/util/typeutil" ) type requestScheduler struct { diff --git a/internal/proxy/server_test.go b/internal/proxy/server_test.go index b85fd58e2b..154d0aea7d 100644 --- a/internal/proxy/server_test.go +++ b/internal/proxy/server_test.go @@ -4,6 +4,12 @@ import ( "context" "encoding/binary" "encoding/json" + "net" + "sort" + "testing" + "time" + "unsafe" + "github.com/apache/pulsar-client-go/pulsar" "github.com/golang/protobuf/proto" "github.com/stretchr/testify/assert" @@ -13,11 +19,6 @@ import ( "github.com/zilliztech/milvus-distributed/internal/util/typeutil" "go.etcd.io/etcd/clientv3" "google.golang.org/grpc" - "net" - "sort" - "testing" - "time" - "unsafe" ) type testMasterServer struct { @@ -129,11 +130,11 @@ func TestProxyServer_WatchEtcd(t *testing.T) { } seg2 := etcdpb.SegmentMeta{ SegmentId: 2, - NumRows: 10, + NumRows: 10, } seg3 := etcdpb.SegmentMeta{ SegmentId: 3, - NumRows: 10, + NumRows: 10, } if cb1, err := json.Marshal(&col1); err != nil { t.Fatal(err) @@ -178,7 +179,7 @@ func TestProxyServer_WatchEtcd(t *testing.T) { } seg5 := etcdpb.SegmentMeta{ SegmentId: 5, - NumRows: 10, + NumRows: 10, } if cb4, err := json.Marshal(&col4); err != nil { t.Fatal(err) diff --git a/internal/proxy/task.go b/internal/proxy/task.go new file mode 100644 index 0000000000..56f4b1f2d5 --- /dev/null +++ b/internal/proxy/task.go @@ -0,0 +1,43 @@ +package proxy + +import ( + "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" + "github.com/zilliztech/milvus-distributed/internal/util/typeutil" +) + +//type TimeStamp uint64 + +type task interface { + Id() int64 // return ReqId + Type() internalpb.ReqType + GetTs() typeutil.Timestamp + SetTs(ts typeutil.Timestamp) + PreExecute() error + Execute() error + PostExecute() error + WaitToFinish() error + Notify() error +} + +type baseTask struct { + ReqType internalpb.ReqType + ReqId int64 + Ts typeutil.Timestamp + ProxyId int64 +} + +func (bt *baseTask) Id() int64 { + return bt.ReqId +} + +func (bt *baseTask) Type() internalpb.ReqType { + return bt.ReqType +} + +func (bt *baseTask) GetTs() typeutil.Timestamp { + return bt.Ts +} + +func (bt *baseTask) SetTs(ts typeutil.Timestamp) { + bt.Ts = ts +} diff --git a/internal/proxy/task_scheduler.go b/internal/proxy/task_scheduler.go new file mode 100644 index 0000000000..6edb23cbe9 --- /dev/null +++ b/internal/proxy/task_scheduler.go @@ -0,0 +1,227 @@ +package proxy + +import ( + "container/list" + "log" + "sync" + + "github.com/zilliztech/milvus-distributed/internal/util/typeutil" +) + +type baseTaskQueue struct { + unissuedTasks *list.List + activeTasks map[typeutil.Timestamp]*task + utLock sync.Mutex + atLock sync.Mutex +} + +type ddTaskQueue struct { + baseTaskQueue + lock sync.Mutex +} + +type dmTaskQueue struct { + baseTaskQueue +} + +type dqTaskQueue struct { + baseTaskQueue +} + +func (queue *baseTaskQueue) Empty() bool { + queue.utLock.Lock() + defer queue.utLock.Unlock() + queue.atLock.Lock() + defer queue.atLock.Unlock() + return queue.unissuedTasks.Len() <= 0 && len(queue.activeTasks) <= 0 +} + +func (queue *baseTaskQueue) AddUnissuedTask(t *task) { + queue.utLock.Lock() + defer queue.utLock.Unlock() + queue.unissuedTasks.PushBack(t) +} + +func (queue *baseTaskQueue) FrontUnissuedTask() *task { + queue.utLock.Lock() + defer queue.utLock.Unlock() + if queue.unissuedTasks.Len() <= 0 { + log.Fatal("sorry, but the unissued task list is empty!") + return nil + } + return queue.unissuedTasks.Front().Value.(*task) +} + +func (queue *baseTaskQueue) PopUnissuedTask() *task { + queue.utLock.Lock() + defer queue.utLock.Unlock() + if queue.unissuedTasks.Len() <= 0 { + log.Fatal("sorry, but the unissued task list is empty!") + return nil + } + ft := queue.unissuedTasks.Front() + return queue.unissuedTasks.Remove(ft).(*task) +} + +func (queue *baseTaskQueue) AddActiveTask(t *task) { + queue.atLock.Lock() + defer queue.atLock.Lock() + ts := (*t).GetTs() + _, ok := queue.activeTasks[ts] + if ok { + log.Fatalf("task with timestamp %d already in active task list!", ts) + } + queue.activeTasks[ts] = t +} + +func (queue *baseTaskQueue) PopActiveTask(ts typeutil.Timestamp) *task { + queue.atLock.Lock() + defer queue.atLock.Lock() + t, ok := queue.activeTasks[ts] + if ok { + delete(queue.activeTasks, ts) + return t + } + log.Fatalf("sorry, but the timestamp %d was not found in the active task list!", ts) + return nil +} + +func (queue *baseTaskQueue) TaskDoneTest(ts typeutil.Timestamp) bool { + queue.utLock.Lock() + defer queue.utLock.Unlock() + for e := queue.unissuedTasks.Front(); e != nil; e = e.Next() { + if (*(e.Value.(*task))).GetTs() >= ts { + return false + } + } + + queue.atLock.Lock() + defer queue.atLock.Unlock() + for ats := range queue.activeTasks { + if ats >= ts { + return false + } + } + + return true +} + +func (queue *ddTaskQueue) Enqueue(t *task) error { + queue.lock.Lock() + defer queue.lock.Unlock() + // TODO: set Ts, ReqId, ProxyId + queue.AddUnissuedTask(t) + return nil +} + +func (queue *dmTaskQueue) Enqueue(t *task) error { + // TODO: set Ts, ReqId, ProxyId + queue.AddUnissuedTask(t) + return nil +} + +func (queue *dqTaskQueue) Enqueue(t *task) error { + // TODO: set Ts, ReqId, ProxyId + queue.AddUnissuedTask(t) + return nil +} + +type taskScheduler struct { + DdQueue *ddTaskQueue + DmQueue *dmTaskQueue + DqQueue *dqTaskQueue + + // tsAllocator, ReqIdAllocator +} + +func (sched *taskScheduler) scheduleDdTask() *task { + return sched.DdQueue.PopUnissuedTask() +} + +func (sched *taskScheduler) scheduleDmTask() *task { + return sched.DmQueue.PopUnissuedTask() +} + +func (sched *taskScheduler) scheduleDqTask() *task { + return sched.DqQueue.PopUnissuedTask() +} + +func (sched *taskScheduler) Start() error { + go func() { + for { + if sched.DdQueue.Empty() { + continue + } + t := sched.scheduleDdTask() + if err := (*t).PreExecute(); err != nil { + return + } + if err := (*t).Execute(); err != nil { + return + } + if err := (*t).PostExecute(); err != nil { + return + } + if err := (*t).WaitToFinish(); err != nil { + return + } + if err := (*t).Notify(); err != nil { + return + } + } + }() + go func() { + for { + if sched.DdQueue.Empty() { + continue + } + t := sched.scheduleDmTask() + if err := (*t).PreExecute(); err != nil { + return + } + if err := (*t).Execute(); err != nil { + return + } + if err := (*t).PostExecute(); err != nil { + return + } + if err := (*t).WaitToFinish(); err != nil { + return + } + if err := (*t).Notify(); err != nil { + return + } + } + }() + go func() { + for { + if sched.DdQueue.Empty() { + continue + } + t := sched.scheduleDqTask() + if err := (*t).PreExecute(); err != nil { + return + } + if err := (*t).Execute(); err != nil { + return + } + if err := (*t).PostExecute(); err != nil { + return + } + if err := (*t).WaitToFinish(); err != nil { + return + } + if err := (*t).Notify(); err != nil { + return + } + } + }() + return nil +} + +func (sched *taskScheduler) TaskDoneTest(ts typeutil.Timestamp) bool { + ddTaskDone := sched.DdQueue.TaskDoneTest(ts) + dmTaskDone := sched.DmQueue.TaskDoneTest(ts) + dqTaskDone := sched.DqQueue.TaskDoneTest(ts) + return ddTaskDone && dmTaskDone && dqTaskDone +} diff --git a/internal/proxy/timetick.go b/internal/proxy/timetick.go index 660ba4f636..10dc86ac17 100644 --- a/internal/proxy/timetick.go +++ b/internal/proxy/timetick.go @@ -2,13 +2,14 @@ package proxy import ( "context" - "github.com/apache/pulsar-client-go/pulsar" - "github.com/zilliztech/milvus-distributed/internal/errors" - pb "github.com/zilliztech/milvus-distributed/internal/proto/message" - "github.com/golang/protobuf/proto" - "github.com/zilliztech/milvus-distributed/internal/util/typeutil" "log" "time" + + "github.com/apache/pulsar-client-go/pulsar" + "github.com/golang/protobuf/proto" + "github.com/zilliztech/milvus-distributed/internal/errors" + pb "github.com/zilliztech/milvus-distributed/internal/proto/message" + "github.com/zilliztech/milvus-distributed/internal/util/typeutil" ) type timeTick struct { @@ -49,10 +50,10 @@ func (tt *timeTick) tick() error { return nil } -func (tt *timeTick) Restart() error{ +func (tt *timeTick) Restart() error { tt.lastTick = 0 ts, err := tt.getTimestamp() - if err != nil{ + if err != nil { return err } diff --git a/internal/proxy/timetick_test.go b/internal/proxy/timetick_test.go index a32b2b21df..c6b860d81f 100644 --- a/internal/proxy/timetick_test.go +++ b/internal/proxy/timetick_test.go @@ -2,21 +2,22 @@ package proxy import ( "context" + "testing" + "time" + "github.com/apache/pulsar-client-go/pulsar" "github.com/golang/protobuf/proto" "github.com/stretchr/testify/assert" pb "github.com/zilliztech/milvus-distributed/internal/proto/message" "github.com/zilliztech/milvus-distributed/internal/util/typeutil" - "testing" - "time" ) func TestTimeTick(t *testing.T) { client, err := pulsar.NewClient(pulsar.ClientOptions{URL: "pulsar://localhost:6650"}) - assert.Nil(t,err) + assert.Nil(t, err) producer, err := client.CreateProducer(pulsar.ProducerOptions{Topic: "timesync"}) - assert.Nil(t,err) + assert.Nil(t, err) consumer, err := client.Subscribe(pulsar.ConsumerOptions{ Topic: "timesync", @@ -24,7 +25,7 @@ func TestTimeTick(t *testing.T) { Type: pulsar.KeyShared, SubscriptionInitialPosition: pulsar.SubscriptionPositionEarliest, }) - assert.Nil(t,err) + assert.Nil(t, err) ctx, _ := context.WithTimeout(context.Background(), 4*time.Second) diff --git a/internal/reader/collection_test.go b/internal/reader/collection_test.go index 8ceb92b3ad..a14cee4271 100644 --- a/internal/reader/collection_test.go +++ b/internal/reader/collection_test.go @@ -2,8 +2,9 @@ package reader import ( "context" - "github.com/stretchr/testify/assert" "testing" + + "github.com/stretchr/testify/assert" ) func TestCollection_NewPartition(t *testing.T) { diff --git a/internal/reader/manipulation_service.go b/internal/reader/manipulation_service.go index 5c79245f22..0a07adad0a 100644 --- a/internal/reader/manipulation_service.go +++ b/internal/reader/manipulation_service.go @@ -3,10 +3,11 @@ package reader import ( "context" "fmt" - msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message" - "github.com/zilliztech/milvus-distributed/internal/util/flowgraph" "log" "sync" + + msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message" + "github.com/zilliztech/milvus-distributed/internal/util/flowgraph" ) type manipulationService struct { diff --git a/internal/reader/manipulation_service_test.go b/internal/reader/manipulation_service_test.go index efaa418eac..45980e49d2 100644 --- a/internal/reader/manipulation_service_test.go +++ b/internal/reader/manipulation_service_test.go @@ -3,15 +3,16 @@ package reader import ( "context" "encoding/binary" - "github.com/stretchr/testify/assert" - "github.com/zilliztech/milvus-distributed/internal/conf" - "github.com/zilliztech/milvus-distributed/internal/msgclient" - msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message" "math" "strconv" "sync" "testing" "time" + + "github.com/stretchr/testify/assert" + "github.com/zilliztech/milvus-distributed/internal/conf" + "github.com/zilliztech/milvus-distributed/internal/msgclient" + msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message" ) func TestInsertAndDelete_MessagesPreprocess(t *testing.T) { diff --git a/internal/reader/message.go b/internal/reader/message.go index a100152063..4e87589c39 100644 --- a/internal/reader/message.go +++ b/internal/reader/message.go @@ -38,7 +38,7 @@ type insertMsg struct { type deletePreprocessMsg struct { deletePreprocessData DeletePreprocessData - timeRange TimeRange + timeRange TimeRange } type deleteMsg struct { @@ -47,7 +47,7 @@ type deleteMsg struct { } type serviceTimeMsg struct { - timeRange TimeRange + timeRange TimeRange } type InsertData struct { diff --git a/internal/reader/meta_test.go b/internal/reader/meta_test.go index 1b5eaa5566..c001f8ac74 100644 --- a/internal/reader/meta_test.go +++ b/internal/reader/meta_test.go @@ -2,17 +2,18 @@ package reader import ( "context" + "log" + "math" + "sync" + "testing" + "time" + "github.com/stretchr/testify/assert" "github.com/zilliztech/milvus-distributed/internal/conf" "github.com/zilliztech/milvus-distributed/internal/master/collection" "github.com/zilliztech/milvus-distributed/internal/master/segment" "github.com/zilliztech/milvus-distributed/internal/msgclient" "github.com/zilliztech/milvus-distributed/internal/proto/schemapb" - "log" - "math" - "sync" - "testing" - "time" ) func TestMeta_GetCollectionObjId(t *testing.T) { diff --git a/internal/reader/partition_test.go b/internal/reader/partition_test.go index 562565a6f9..d6c44871f2 100644 --- a/internal/reader/partition_test.go +++ b/internal/reader/partition_test.go @@ -2,8 +2,9 @@ package reader import ( "context" - "github.com/stretchr/testify/assert" "testing" + + "github.com/stretchr/testify/assert" ) func TestPartition_NewSegment(t *testing.T) { diff --git a/internal/reader/query_node_time_test.go b/internal/reader/query_node_time_test.go index 57d179a103..ec19694b9f 100644 --- a/internal/reader/query_node_time_test.go +++ b/internal/reader/query_node_time_test.go @@ -1,8 +1,9 @@ package reader import ( - "github.com/stretchr/testify/assert" "testing" + + "github.com/stretchr/testify/assert" ) func TestQueryNodeTime_UpdateReadTimeSync(t *testing.T) { diff --git a/internal/reader/quety_node_test.go b/internal/reader/quety_node_test.go index 0df11338c3..09eaf56f4e 100644 --- a/internal/reader/quety_node_test.go +++ b/internal/reader/quety_node_test.go @@ -2,9 +2,10 @@ package reader import ( "context" + "testing" + "github.com/stretchr/testify/assert" "github.com/zilliztech/milvus-distributed/internal/conf" - "testing" ) func TestQueryNode_CreateQueryNode(t *testing.T) { diff --git a/internal/reader/reader.go b/internal/reader/reader.go index 9d5609f398..2a6487627a 100644 --- a/internal/reader/reader.go +++ b/internal/reader/reader.go @@ -3,10 +3,11 @@ package reader import ( "context" "fmt" - "github.com/stretchr/testify/assert" - "github.com/zilliztech/milvus-distributed/internal/msgclient" "log" "sync" + + "github.com/stretchr/testify/assert" + "github.com/zilliztech/milvus-distributed/internal/msgclient" ) func StartQueryNode(ctx context.Context, pulsarURL string) { diff --git a/internal/reader/reader_test.go b/internal/reader/reader_test.go index 5580ee9c43..39b9ae757a 100644 --- a/internal/reader/reader_test.go +++ b/internal/reader/reader_test.go @@ -2,13 +2,14 @@ package reader import ( "context" - "github.com/stretchr/testify/assert" - "github.com/zilliztech/milvus-distributed/internal/conf" - "github.com/zilliztech/milvus-distributed/internal/msgclient" "strconv" "sync" "testing" "time" + + "github.com/stretchr/testify/assert" + "github.com/zilliztech/milvus-distributed/internal/conf" + "github.com/zilliztech/milvus-distributed/internal/msgclient" ) const ctxTimeInMillisecond = 10 diff --git a/internal/reader/result.go b/internal/reader/result.go index 17610e5dcb..9a0bfb3fbc 100644 --- a/internal/reader/result.go +++ b/internal/reader/result.go @@ -2,8 +2,9 @@ package reader import ( "context" - "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" + "github.com/zilliztech/milvus-distributed/internal/proto/commonpb" + "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message" ) @@ -23,8 +24,7 @@ func (node *QueryNode) PublishSearchResult(results *msgpb.QueryResult) commonpb. } func (node *QueryNode) PublishFailedSearchResult() commonpb.Status { - var results = msgpb.QueryResult{ - } + var results = msgpb.QueryResult{} var ctx = context.Background() diff --git a/internal/reader/result_test.go b/internal/reader/result_test.go index 3e0dce67e1..081873f5bb 100644 --- a/internal/reader/result_test.go +++ b/internal/reader/result_test.go @@ -2,13 +2,14 @@ package reader import ( "context" - "github.com/stretchr/testify/assert" - "github.com/zilliztech/milvus-distributed/internal/conf" - "github.com/zilliztech/milvus-distributed/internal/msgclient" "strconv" "testing" "time" + "github.com/stretchr/testify/assert" + "github.com/zilliztech/milvus-distributed/internal/conf" + "github.com/zilliztech/milvus-distributed/internal/msgclient" + //masterPb "github.com/zilliztech/milvus-distributed/internal/proto/masterpb" "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message" diff --git a/internal/reader/search.go b/internal/reader/search.go index 0a69b5a2de..8c4934ca63 100644 --- a/internal/reader/search.go +++ b/internal/reader/search.go @@ -2,9 +2,10 @@ package reader import ( "fmt" + "sort" + "github.com/zilliztech/milvus-distributed/internal/conf" msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message" - "sort" ) func (node *QueryNode) Search(searchMessages []*msgPb.SearchMsg) msgPb.Status { diff --git a/internal/reader/search_test.go b/internal/reader/search_test.go index 080f75afbd..be06c215a0 100644 --- a/internal/reader/search_test.go +++ b/internal/reader/search_test.go @@ -3,14 +3,15 @@ package reader import ( "context" "encoding/binary" - "github.com/stretchr/testify/assert" - "github.com/zilliztech/milvus-distributed/internal/conf" - "github.com/zilliztech/milvus-distributed/internal/msgclient" - msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message" "math" "strconv" "sync" "testing" + + "github.com/stretchr/testify/assert" + "github.com/zilliztech/milvus-distributed/internal/conf" + "github.com/zilliztech/milvus-distributed/internal/msgclient" + msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message" ) // NOTE: start pulsar before test diff --git a/internal/reader/segment.go b/internal/reader/segment.go index e892c9164a..5685a4f1be 100644 --- a/internal/reader/segment.go +++ b/internal/reader/segment.go @@ -13,11 +13,12 @@ package reader */ import "C" import ( + "strconv" + "unsafe" + "github.com/stretchr/testify/assert" "github.com/zilliztech/milvus-distributed/internal/errors" msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message" - "strconv" - "unsafe" ) const SegmentLifetime = 20000 diff --git a/internal/reader/segment_service.go b/internal/reader/segment_service.go index 544f48d9c3..9e5d241a60 100644 --- a/internal/reader/segment_service.go +++ b/internal/reader/segment_service.go @@ -2,11 +2,12 @@ package reader import ( "fmt" - "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" - "github.com/zilliztech/milvus-distributed/internal/proto/commonpb" "log" "strconv" "time" + + "github.com/zilliztech/milvus-distributed/internal/proto/commonpb" + "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" ) //func (node *QueryNode) SegmentsManagement() { @@ -58,7 +59,7 @@ func (node *QueryNode) SegmentStatistic(sleepMillisecondTime int) { // TODO: set master pb's segment id type from uint64 to int64 SegmentId: segmentID, MemorySize: currentMemSize, - NumRows: segmentNumOfRows, + NumRows: segmentNumOfRows, } statisticData = append(statisticData, stat) diff --git a/internal/reader/segment_service_test.go b/internal/reader/segment_service_test.go index 83ac389d6e..0cde5da7aa 100644 --- a/internal/reader/segment_service_test.go +++ b/internal/reader/segment_service_test.go @@ -2,11 +2,12 @@ package reader import ( "context" - "github.com/zilliztech/milvus-distributed/internal/conf" - "github.com/zilliztech/milvus-distributed/internal/msgclient" "strconv" "testing" "time" + + "github.com/zilliztech/milvus-distributed/internal/conf" + "github.com/zilliztech/milvus-distributed/internal/msgclient" ) //func TestSegmentManagement_SegmentsManagement(t *testing.T) { diff --git a/internal/reader/util_functions.go b/internal/reader/util_functions.go index e6c1c77b24..5ffaf4ac0f 100644 --- a/internal/reader/util_functions.go +++ b/internal/reader/util_functions.go @@ -4,10 +4,11 @@ import ( "encoding/json" "errors" "fmt" - log "github.com/apache/pulsar/pulsar-client-go/logutil" "os" "strconv" "time" + + log "github.com/apache/pulsar/pulsar-client-go/logutil" ) // Function `GetSegmentByEntityId` should return entityIDs, timestamps and segmentIDs diff --git a/internal/reader/util_functions_test.go b/internal/reader/util_functions_test.go index 17226061a1..bf193dccea 100644 --- a/internal/reader/util_functions_test.go +++ b/internal/reader/util_functions_test.go @@ -2,13 +2,14 @@ package reader import ( "context" - "github.com/zilliztech/milvus-distributed/internal/conf" - "github.com/zilliztech/milvus-distributed/internal/msgclient" - msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message" "strconv" "testing" "time" + "github.com/zilliztech/milvus-distributed/internal/conf" + "github.com/zilliztech/milvus-distributed/internal/msgclient" + msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message" + "github.com/stretchr/testify/assert" ) diff --git a/internal/storage/internal/S3/S3_test.go b/internal/storage/internal/S3/S3_test.go index 2c6da88cbb..e99330b849 100644 --- a/internal/storage/internal/S3/S3_test.go +++ b/internal/storage/internal/S3/S3_test.go @@ -2,15 +2,15 @@ package S3_driver_test import ( "context" - s3_driver "github.com/zilliztech/milvus-distributed/internal/storage/internal/S3" - "github.com/stretchr/testify/assert" "testing" + + "github.com/stretchr/testify/assert" + s3_driver "github.com/zilliztech/milvus-distributed/internal/storage/internal/S3" ) var ctx = context.Background() var client, err = s3_driver.NewS3Driver(ctx) - func TestS3Driver_PutRowAndGetRow(t *testing.T) { err = client.PutRow(ctx, []byte("bar"), []byte("abcdefghijklmnoopqrstuvwxyz"), "SegmentA", 1) assert.Nil(t, err) @@ -30,7 +30,7 @@ func TestS3Driver_PutRowAndGetRow(t *testing.T) { assert.Equal(t, "testkeybarorbar_1", string(object)) } -func TestS3Driver_DeleteRow(t *testing.T){ +func TestS3Driver_DeleteRow(t *testing.T) { err = client.DeleteRow(ctx, []byte("bar"), 5) assert.Nil(t, err) object, _ := client.GetRow(ctx, []byte("bar"), 6) @@ -63,7 +63,7 @@ func TestS3Driver_GetSegments(t *testing.T) { } } -func TestS3Driver_PutRowsAndGetRows(t *testing.T){ +func TestS3Driver_PutRowsAndGetRows(t *testing.T) { keys := [][]byte{[]byte("foo"), []byte("bar")} values := [][]byte{[]byte("The key is foo!"), []byte("The key is bar!")} segments := []string{"segmentA", "segmentB"} @@ -77,7 +77,7 @@ func TestS3Driver_PutRowsAndGetRows(t *testing.T){ assert.Equal(t, "The key is bar!", string(objects[1])) } -func TestS3Driver_DeleteRows(t *testing.T){ +func TestS3Driver_DeleteRows(t *testing.T) { keys := [][]byte{[]byte("foo"), []byte("bar")} timestamps := []uint64{3, 3} err := client.DeleteRows(ctx, keys, timestamps) @@ -92,11 +92,11 @@ func TestS3Driver_DeleteRows(t *testing.T){ func TestS3Driver_PutLogAndGetLog(t *testing.T) { err = client.PutLog(ctx, []byte("insert"), []byte("This is insert log!"), 1, 11) assert.Nil(t, err) - err = client.PutLog(ctx, []byte("delete"), []byte("This is delete log!"), 2, 10) + err = client.PutLog(ctx, []byte("delete"), []byte("This is delete log!"), 2, 10) assert.Nil(t, err) - err = client.PutLog(ctx, []byte("update"), []byte("This is update log!"), 3, 9) + err = client.PutLog(ctx, []byte("update"), []byte("This is update log!"), 3, 9) assert.Nil(t, err) - err = client.PutLog(ctx, []byte("select"), []byte("This is select log!"), 4, 8) + err = client.PutLog(ctx, []byte("select"), []byte("This is select log!"), 4, 8) assert.Nil(t, err) channels := []int{5, 8, 9, 10, 11, 12, 13} @@ -119,7 +119,7 @@ func TestS3Driver_Segment(t *testing.T) { assert.Nil(t, err) } -func TestS3Driver_SegmentDL(t *testing.T){ +func TestS3Driver_SegmentDL(t *testing.T) { err := client.PutSegmentDL(ctx, "segmentB", []byte("This is segmentB's delete log!")) assert.Nil(t, err) @@ -129,4 +129,4 @@ func TestS3Driver_SegmentDL(t *testing.T){ err = client.DeleteSegmentDL(ctx, "segmentB") assert.Nil(t, err) -} \ No newline at end of file +} diff --git a/internal/storage/internal/S3/s3_engine.go b/internal/storage/internal/S3/s3_engine.go index 62caec8ec5..5e8c45a4fc 100644 --- a/internal/storage/internal/S3/s3_engine.go +++ b/internal/storage/internal/S3/s3_engine.go @@ -3,12 +3,13 @@ package S3_driver import ( "bytes" "context" + "io" + "github.com/aws/aws-sdk-go/aws" "github.com/aws/aws-sdk-go/aws/session" "github.com/aws/aws-sdk-go/service/s3" "github.com/zilliztech/milvus-distributed/internal/conf" . "github.com/zilliztech/milvus-distributed/internal/storage/type" - "io" ) var bucketName = conf.Config.Writer.Bucket @@ -49,7 +50,7 @@ func (s *S3Store) Put(ctx context.Context, key Key, value Value) error { func (s *S3Store) Get(ctx context.Context, key Key) (Value, error) { object, err := s.client.GetObjectWithContext(ctx, &s3.GetObjectInput{ Bucket: aws.String(bucketName), - Key: aws.String(string(key)), + Key: aws.String(string(key)), }) if err != nil { return nil, err @@ -85,7 +86,7 @@ func (s *S3Store) GetByPrefix(ctx context.Context, prefix Key, keyOnly bool) ([] objectsValues = append(objectsValues, value) } } - }else { + } else { return nil, nil, err } @@ -102,7 +103,7 @@ func (s *S3Store) Scan(ctx context.Context, keyStart Key, keyEnd Key, limit int, Prefix: aws.String(string(keyStart)), }) if err == nil && objects != nil { - for _, object := range objects.Contents{ + for _, object := range objects.Contents { if *object.Key >= string(keyEnd) { keys = append(keys, []byte(*object.Key)) if !keyOnly { @@ -126,7 +127,7 @@ func (s *S3Store) Scan(ctx context.Context, keyStart Key, keyEnd Key, limit int, func (s *S3Store) Delete(ctx context.Context, key Key) error { _, err := s.client.DeleteObjectWithContext(ctx, &s3.DeleteObjectInput{ Bucket: aws.String(bucketName), - Key: aws.String(string(key)), + Key: aws.String(string(key)), }) return err } @@ -142,7 +143,7 @@ func (s *S3Store) DeleteByPrefix(ctx context.Context, prefix Key) error { for _, object := range objects.Contents { _, err := s.client.DeleteObjectWithContext(ctx, &s3.DeleteObjectInput{ Bucket: aws.String(bucketName), - Key: object.Key, + Key: object.Key, }) return err } @@ -160,10 +161,10 @@ func (s *S3Store) DeleteRange(ctx context.Context, keyStart Key, keyEnd Key) err if objects != nil && err == nil { for _, object := range objects.Contents { - if *object.Key > string(keyEnd){ + if *object.Key > string(keyEnd) { _, err := s.client.DeleteObjectWithContext(ctx, &s3.DeleteObjectInput{ Bucket: aws.String(bucketName), - Key: object.Key, + Key: object.Key, }) return err } diff --git a/internal/storage/internal/S3/s3_store.go b/internal/storage/internal/S3/s3_store.go index 10d6adda02..63341d69bb 100644 --- a/internal/storage/internal/S3/s3_store.go +++ b/internal/storage/internal/S3/s3_store.go @@ -2,6 +2,7 @@ package S3_driver import ( "context" + "github.com/aws/aws-sdk-go/aws" "github.com/aws/aws-sdk-go/aws/endpoints" "github.com/zilliztech/milvus-distributed/internal/storage/internal/minio/codec" @@ -115,25 +116,25 @@ func (s *S3Driver) GetRow(ctx context.Context, key Key, timestamp Timestamp) (Va } keys, values, err := s.driver.Scan(ctx, append(key, byte('_')), minioKey, 1, false) - if values == nil || keys == nil{ + if values == nil || keys == nil { return nil, err } _, _, suffix, err := codec.MvccDecode(keys[0]) - if err != nil{ + if err != nil { return nil, err } - if suffix == "delete"{ + if suffix == "delete" { return nil, nil } return values[0], err } -func (s *S3Driver) GetRows(ctx context.Context, keys []Key, timestamps []Timestamp) ([]Value, error){ +func (s *S3Driver) GetRows(ctx context.Context, keys []Key, timestamps []Timestamp) ([]Value, error) { var values []Value - for i, key := range keys{ + for i, key := range keys { value, err := s.GetRow(ctx, key, timestamps[i]) - if err!= nil{ + if err != nil { return nil, err } values = append(values, value) @@ -141,32 +142,32 @@ func (s *S3Driver) GetRows(ctx context.Context, keys []Key, timestamps []Timesta return values, nil } -func (s *S3Driver) PutRow(ctx context.Context, key Key, value Value, segment string, timestamp Timestamp) error{ +func (s *S3Driver) PutRow(ctx context.Context, key Key, value Value, segment string, timestamp Timestamp) error { minioKey, err := codec.MvccEncode(key, timestamp, segment) - if err != nil{ + if err != nil { return err } err = s.driver.Put(ctx, minioKey, value) return err } -func (s *S3Driver) PutRows(ctx context.Context, keys []Key, values []Value, segments []string, timestamps []Timestamp) error{ +func (s *S3Driver) PutRows(ctx context.Context, keys []Key, values []Value, segments []string, timestamps []Timestamp) error { maxThread := 100 batchSize := 1 keysLength := len(keys) - if keysLength / batchSize > maxThread { + if keysLength/batchSize > maxThread { batchSize = keysLength / maxThread } batchNums := keysLength / batchSize - if keysLength % batchSize != 0 { - batchNums = keysLength / batchSize + 1 + if keysLength%batchSize != 0 { + batchNums = keysLength/batchSize + 1 } errCh := make(chan error) f := func(ctx2 context.Context, keys2 []Key, values2 []Value, segments2 []string, timestamps2 []Timestamp) { - for i := 0; i < len(keys2); i++{ + for i := 0; i < len(keys2); i++ { err := s.PutRow(ctx2, keys2[i], values2[i], segments2[i], timestamps2[i]) errCh <- err } @@ -183,19 +184,19 @@ func (s *S3Driver) PutRows(ctx context.Context, keys []Key, values []Value, segm } for i := 0; i < len(keys); i++ { - if err := <- errCh; err != nil { + if err := <-errCh; err != nil { return err } } return nil } -func (s *S3Driver) GetSegments(ctx context.Context, key Key, timestamp Timestamp) ([]string, error){ +func (s *S3Driver) GetSegments(ctx context.Context, key Key, timestamp Timestamp) ([]string, error) { keyEnd, err := codec.MvccEncode(key, timestamp, "") - if err != nil{ + if err != nil { return nil, err } - keys, _, err := s.driver.Scan(ctx, append(key, byte('_')), keyEnd, -1,true) + keys, _, err := s.driver.Scan(ctx, append(key, byte('_')), keyEnd, -1, true) if err != nil { return nil, err } @@ -219,9 +220,9 @@ func (s *S3Driver) GetSegments(ctx context.Context, key Key, timestamp Timestamp return segments, err } -func (s *S3Driver) DeleteRow(ctx context.Context, key Key, timestamp Timestamp) error{ +func (s *S3Driver) DeleteRow(ctx context.Context, key Key, timestamp Timestamp) error { minioKey, err := codec.MvccEncode(key, timestamp, "delete") - if err != nil{ + if err != nil { return err } value := []byte("0") @@ -229,24 +230,24 @@ func (s *S3Driver) DeleteRow(ctx context.Context, key Key, timestamp Timestamp) return err } -func (s *S3Driver) DeleteRows(ctx context.Context, keys []Key, timestamps []Timestamp) error{ +func (s *S3Driver) DeleteRows(ctx context.Context, keys []Key, timestamps []Timestamp) error { maxThread := 100 batchSize := 1 keysLength := len(keys) - if keysLength / batchSize > maxThread { + if keysLength/batchSize > maxThread { batchSize = keysLength / maxThread } batchNums := keysLength / batchSize - if keysLength % batchSize != 0 { - batchNums = keysLength / batchSize + 1 + if keysLength%batchSize != 0 { + batchNums = keysLength/batchSize + 1 } errCh := make(chan error) f := func(ctx2 context.Context, keys2 []Key, timestamps2 []Timestamp) { - for i := 0; i < len(keys2); i++{ + for i := 0; i < len(keys2); i++ { err := s.DeleteRow(ctx2, keys2[i], timestamps2[i]) errCh <- err } @@ -263,14 +264,14 @@ func (s *S3Driver) DeleteRows(ctx context.Context, keys []Key, timestamps []Time } for i := 0; i < len(keys); i++ { - if err := <- errCh; err != nil { + if err := <-errCh; err != nil { return err } } return nil } -func (s *S3Driver) PutLog(ctx context.Context, key Key, value Value, timestamp Timestamp, channel int) error{ +func (s *S3Driver) PutLog(ctx context.Context, key Key, value Value, timestamp Timestamp, channel int) error { logKey := codec.LogEncode(key, timestamp, channel) err := s.driver.Put(ctx, logKey, value) return err @@ -283,12 +284,12 @@ func (s *S3Driver) GetLog(ctx context.Context, start Timestamp, end Timestamp, c } var resultValues []Value - for i, key := range keys{ + for i, key := range keys { _, ts, channel, err := codec.LogDecode(string(key)) if err != nil { return nil, err } - if ts >= start && ts <= end { + if ts >= start && ts <= end { for j := 0; j < len(channels); j++ { if channel == channels[j] { resultValues = append(resultValues, values[i]) @@ -300,32 +301,32 @@ func (s *S3Driver) GetLog(ctx context.Context, start Timestamp, end Timestamp, c return resultValues, nil } -func (s *S3Driver) GetSegmentIndex(ctx context.Context, segment string) (SegmentIndex, error){ +func (s *S3Driver) GetSegmentIndex(ctx context.Context, segment string) (SegmentIndex, error) { return s.driver.Get(ctx, codec.SegmentEncode(segment, "index")) } -func (s *S3Driver) PutSegmentIndex(ctx context.Context, segment string, index SegmentIndex) error{ +func (s *S3Driver) PutSegmentIndex(ctx context.Context, segment string, index SegmentIndex) error { return s.driver.Put(ctx, codec.SegmentEncode(segment, "index"), index) } -func (s *S3Driver) DeleteSegmentIndex(ctx context.Context, segment string) error{ +func (s *S3Driver) DeleteSegmentIndex(ctx context.Context, segment string) error { return s.driver.Delete(ctx, codec.SegmentEncode(segment, "index")) } -func (s *S3Driver) GetSegmentDL(ctx context.Context, segment string) (SegmentDL, error){ +func (s *S3Driver) GetSegmentDL(ctx context.Context, segment string) (SegmentDL, error) { return s.driver.Get(ctx, codec.SegmentEncode(segment, "DL")) } -func (s *S3Driver) PutSegmentDL(ctx context.Context, segment string, log SegmentDL) error{ +func (s *S3Driver) PutSegmentDL(ctx context.Context, segment string, log SegmentDL) error { return s.driver.Put(ctx, codec.SegmentEncode(segment, "DL"), log) } -func (s *S3Driver) DeleteSegmentDL(ctx context.Context, segment string) error{ +func (s *S3Driver) DeleteSegmentDL(ctx context.Context, segment string) error { return s.driver.Delete(ctx, codec.SegmentEncode(segment, "DL")) } diff --git a/internal/storage/internal/minio/codec/codec.go b/internal/storage/internal/minio/codec/codec.go index 284b91d009..4d2b76ee2f 100644 --- a/internal/storage/internal/minio/codec/codec.go +++ b/internal/storage/internal/minio/codec/codec.go @@ -46,7 +46,7 @@ func MvccDecode(key []byte) (string, uint64, string, error) { func LogEncode(key []byte, ts uint64, channel int) []byte { suffix := string(key) + "_" + fmt.Sprintf("%d", channel) logKey, err := MvccEncode([]byte("log"), ts, suffix) - if err != nil{ + if err != nil { return nil } return logKey diff --git a/internal/storage/internal/minio/minio_store.go b/internal/storage/internal/minio/minio_store.go index 850220dad3..7181c46809 100644 --- a/internal/storage/internal/minio/minio_store.go +++ b/internal/storage/internal/minio/minio_store.go @@ -2,11 +2,12 @@ package minio_driver import ( "context" + + "github.com/minio/minio-go/v7" + "github.com/minio/minio-go/v7/pkg/credentials" "github.com/zilliztech/milvus-distributed/internal/storage/internal/minio/codec" . "github.com/zilliztech/milvus-distributed/internal/storage/internal/minio/codec" . "github.com/zilliztech/milvus-distributed/internal/storage/type" - "github.com/minio/minio-go/v7" - "github.com/minio/minio-go/v7/pkg/credentials" ) type minioDriver struct { @@ -135,25 +136,25 @@ func (s *minioDriver) GetRow(ctx context.Context, key Key, timestamp Timestamp) } keys, values, err := s.driver.Scan(ctx, append(key, byte('_')), minioKey, 1, false) - if values == nil || keys == nil{ + if values == nil || keys == nil { return nil, err } _, _, suffix, err := MvccDecode(keys[0]) - if err != nil{ + if err != nil { return nil, err } - if suffix == "delete"{ + if suffix == "delete" { return nil, nil } return values[0], err } -func (s *minioDriver) GetRows(ctx context.Context, keys []Key, timestamps []Timestamp) ([]Value, error){ +func (s *minioDriver) GetRows(ctx context.Context, keys []Key, timestamps []Timestamp) ([]Value, error) { var values []Value - for i, key := range keys{ + for i, key := range keys { value, err := s.GetRow(ctx, key, timestamps[i]) - if err!= nil{ + if err != nil { return nil, err } values = append(values, value) @@ -161,32 +162,32 @@ func (s *minioDriver) GetRows(ctx context.Context, keys []Key, timestamps []Time return values, nil } -func (s *minioDriver) PutRow(ctx context.Context, key Key, value Value, segment string, timestamp Timestamp) error{ +func (s *minioDriver) PutRow(ctx context.Context, key Key, value Value, segment string, timestamp Timestamp) error { minioKey, err := MvccEncode(key, timestamp, segment) - if err != nil{ + if err != nil { return err } err = s.driver.Put(ctx, minioKey, value) return err } -func (s *minioDriver) PutRows(ctx context.Context, keys []Key, values []Value, segments []string, timestamps []Timestamp) error{ +func (s *minioDriver) PutRows(ctx context.Context, keys []Key, values []Value, segments []string, timestamps []Timestamp) error { maxThread := 100 batchSize := 1 keysLength := len(keys) - if keysLength / batchSize > maxThread { + if keysLength/batchSize > maxThread { batchSize = keysLength / maxThread } batchNums := keysLength / batchSize - if keysLength % batchSize != 0 { - batchNums = keysLength / batchSize + 1 + if keysLength%batchSize != 0 { + batchNums = keysLength/batchSize + 1 } errCh := make(chan error) f := func(ctx2 context.Context, keys2 []Key, values2 []Value, segments2 []string, timestamps2 []Timestamp) { - for i := 0; i < len(keys2); i++{ + for i := 0; i < len(keys2); i++ { err := s.PutRow(ctx2, keys2[i], values2[i], segments2[i], timestamps2[i]) errCh <- err } @@ -203,19 +204,19 @@ func (s *minioDriver) PutRows(ctx context.Context, keys []Key, values []Value, s } for i := 0; i < len(keys); i++ { - if err := <- errCh; err != nil { + if err := <-errCh; err != nil { return err } } return nil } -func (s *minioDriver) GetSegments(ctx context.Context, key Key, timestamp Timestamp) ([]string, error){ +func (s *minioDriver) GetSegments(ctx context.Context, key Key, timestamp Timestamp) ([]string, error) { keyEnd, err := MvccEncode(key, timestamp, "") - if err != nil{ + if err != nil { return nil, err } - keys, _, err := s.driver.Scan(ctx, append(key, byte('_')), keyEnd, -1,true) + keys, _, err := s.driver.Scan(ctx, append(key, byte('_')), keyEnd, -1, true) if err != nil { return nil, err } @@ -239,9 +240,9 @@ func (s *minioDriver) GetSegments(ctx context.Context, key Key, timestamp Timest return segments, err } -func (s *minioDriver) DeleteRow(ctx context.Context, key Key, timestamp Timestamp) error{ +func (s *minioDriver) DeleteRow(ctx context.Context, key Key, timestamp Timestamp) error { minioKey, err := MvccEncode(key, timestamp, "delete") - if err != nil{ + if err != nil { return err } value := []byte("0") @@ -249,24 +250,24 @@ func (s *minioDriver) DeleteRow(ctx context.Context, key Key, timestamp Timestam return err } -func (s *minioDriver) DeleteRows(ctx context.Context, keys []Key, timestamps []Timestamp) error{ +func (s *minioDriver) DeleteRows(ctx context.Context, keys []Key, timestamps []Timestamp) error { maxThread := 100 batchSize := 1 keysLength := len(keys) - if keysLength / batchSize > maxThread { + if keysLength/batchSize > maxThread { batchSize = keysLength / maxThread } batchNums := keysLength / batchSize - if keysLength % batchSize != 0 { - batchNums = keysLength / batchSize + 1 + if keysLength%batchSize != 0 { + batchNums = keysLength/batchSize + 1 } errCh := make(chan error) f := func(ctx2 context.Context, keys2 []Key, timestamps2 []Timestamp) { - for i := 0; i < len(keys2); i++{ + for i := 0; i < len(keys2); i++ { err := s.DeleteRow(ctx2, keys2[i], timestamps2[i]) errCh <- err } @@ -283,14 +284,14 @@ func (s *minioDriver) DeleteRows(ctx context.Context, keys []Key, timestamps []T } for i := 0; i < len(keys); i++ { - if err := <- errCh; err != nil { + if err := <-errCh; err != nil { return err } } return nil } -func (s *minioDriver) PutLog(ctx context.Context, key Key, value Value, timestamp Timestamp, channel int) error{ +func (s *minioDriver) PutLog(ctx context.Context, key Key, value Value, timestamp Timestamp, channel int) error { logKey := LogEncode(key, timestamp, channel) err := s.driver.Put(ctx, logKey, value) return err @@ -303,12 +304,12 @@ func (s *minioDriver) GetLog(ctx context.Context, start Timestamp, end Timestamp } var resultValues []Value - for i, key := range keys{ + for i, key := range keys { _, ts, channel, err := LogDecode(string(key)) if err != nil { return nil, err } - if ts >= start && ts <= end { + if ts >= start && ts <= end { for j := 0; j < len(channels); j++ { if channel == channels[j] { resultValues = append(resultValues, values[i]) @@ -320,32 +321,32 @@ func (s *minioDriver) GetLog(ctx context.Context, start Timestamp, end Timestamp return resultValues, nil } -func (s *minioDriver) GetSegmentIndex(ctx context.Context, segment string) (SegmentIndex, error){ +func (s *minioDriver) GetSegmentIndex(ctx context.Context, segment string) (SegmentIndex, error) { return s.driver.Get(ctx, SegmentEncode(segment, "index")) } -func (s *minioDriver) PutSegmentIndex(ctx context.Context, segment string, index SegmentIndex) error{ +func (s *minioDriver) PutSegmentIndex(ctx context.Context, segment string, index SegmentIndex) error { return s.driver.Put(ctx, SegmentEncode(segment, "index"), index) } -func (s *minioDriver) DeleteSegmentIndex(ctx context.Context, segment string) error{ +func (s *minioDriver) DeleteSegmentIndex(ctx context.Context, segment string) error { return s.driver.Delete(ctx, SegmentEncode(segment, "index")) } -func (s *minioDriver) GetSegmentDL(ctx context.Context, segment string) (SegmentDL, error){ +func (s *minioDriver) GetSegmentDL(ctx context.Context, segment string) (SegmentDL, error) { return s.driver.Get(ctx, SegmentEncode(segment, "DL")) } -func (s *minioDriver) PutSegmentDL(ctx context.Context, segment string, log SegmentDL) error{ +func (s *minioDriver) PutSegmentDL(ctx context.Context, segment string, log SegmentDL) error { return s.driver.Put(ctx, SegmentEncode(segment, "DL"), log) } -func (s *minioDriver) DeleteSegmentDL(ctx context.Context, segment string) error{ +func (s *minioDriver) DeleteSegmentDL(ctx context.Context, segment string) error { return s.driver.Delete(ctx, SegmentEncode(segment, "DL")) } diff --git a/internal/storage/internal/minio/minio_storeEngine.go b/internal/storage/internal/minio/minio_storeEngine.go index a9a627eac8..d5650fc82a 100644 --- a/internal/storage/internal/minio/minio_storeEngine.go +++ b/internal/storage/internal/minio/minio_storeEngine.go @@ -3,10 +3,11 @@ package minio_driver import ( "bytes" "context" + "io" + + "github.com/minio/minio-go/v7" "github.com/zilliztech/milvus-distributed/internal/conf" . "github.com/zilliztech/milvus-distributed/internal/storage/type" - "github.com/minio/minio-go/v7" - "io" ) var bucketName = conf.Config.Writer.Bucket @@ -32,7 +33,7 @@ func (s *minioStore) Get(ctx context.Context, key Key) (Value, error) { return nil, err } - size := 256*1024 + size := 256 * 1024 buf := make([]byte, size) n, err := object.Read(buf) if err != nil && err != io.EOF { @@ -49,9 +50,9 @@ func (s *minioStore) GetByPrefix(ctx context.Context, prefix Key, keyOnly bool) for object := range objects { objectsKeys = append(objectsKeys, []byte(object.Key)) - if !keyOnly{ + if !keyOnly { value, err := s.Get(ctx, []byte(object.Key)) - if err != nil{ + if err != nil { return nil, nil, err } objectsValues = append(objectsValues, value) @@ -62,11 +63,11 @@ func (s *minioStore) GetByPrefix(ctx context.Context, prefix Key, keyOnly bool) } -func (s *minioStore) Scan(ctx context.Context, keyStart Key, keyEnd Key, limit int, keyOnly bool) ([]Key, []Value, error){ +func (s *minioStore) Scan(ctx context.Context, keyStart Key, keyEnd Key, limit int, keyOnly bool) ([]Key, []Value, error) { var keys []Key var values []Value limitCount := uint(limit) - for object := range s.client.ListObjects(ctx, bucketName, minio.ListObjectsOptions{Prefix: string(keyStart)}) { + for object := range s.client.ListObjects(ctx, bucketName, minio.ListObjectsOptions{Prefix: string(keyStart)}) { if object.Key >= string(keyEnd) { keys = append(keys, []byte(object.Key)) if !keyOnly { @@ -77,7 +78,7 @@ func (s *minioStore) Scan(ctx context.Context, keyStart Key, keyEnd Key, limit i values = append(values, value) } limitCount-- - if limitCount <= 0{ + if limitCount <= 0 { break } } @@ -91,18 +92,18 @@ func (s *minioStore) Delete(ctx context.Context, key Key) error { return err } -func (s *minioStore) DeleteByPrefix(ctx context.Context, prefix Key) error{ +func (s *minioStore) DeleteByPrefix(ctx context.Context, prefix Key) error { objectsCh := make(chan minio.ObjectInfo) go func() { defer close(objectsCh) - for object := range s.client.ListObjects(ctx, bucketName, minio.ListObjectsOptions{Prefix: string(prefix)}){ + for object := range s.client.ListObjects(ctx, bucketName, minio.ListObjectsOptions{Prefix: string(prefix)}) { objectsCh <- object } }() - for rErr := range s.client.RemoveObjects(ctx, bucketName, objectsCh, minio.RemoveObjectsOptions{GovernanceBypass: true}){ + for rErr := range s.client.RemoveObjects(ctx, bucketName, objectsCh, minio.RemoveObjectsOptions{GovernanceBypass: true}) { if rErr.Err != nil { return rErr.Err } @@ -116,14 +117,14 @@ func (s *minioStore) DeleteRange(ctx context.Context, keyStart Key, keyEnd Key) go func() { defer close(objectsCh) - for object := range s.client.ListObjects(ctx, bucketName, minio.ListObjectsOptions{Prefix: string(keyStart)}){ + for object := range s.client.ListObjects(ctx, bucketName, minio.ListObjectsOptions{Prefix: string(keyStart)}) { if object.Key <= string(keyEnd) { objectsCh <- object } } }() - for rErr := range s.client.RemoveObjects(ctx, bucketName, objectsCh, minio.RemoveObjectsOptions{GovernanceBypass: true}){ + for rErr := range s.client.RemoveObjects(ctx, bucketName, objectsCh, minio.RemoveObjectsOptions{GovernanceBypass: true}) { if rErr.Err != nil { return rErr.Err } diff --git a/internal/storage/internal/minio/minio_test.go b/internal/storage/internal/minio/minio_test.go index c91e047446..2731f37593 100644 --- a/internal/storage/internal/minio/minio_test.go +++ b/internal/storage/internal/minio/minio_test.go @@ -2,15 +2,15 @@ package minio_driver_test import ( "context" - minio_driver "github.com/zilliztech/milvus-distributed/internal/storage/internal/minio" - "github.com/stretchr/testify/assert" "testing" + + "github.com/stretchr/testify/assert" + minio_driver "github.com/zilliztech/milvus-distributed/internal/storage/internal/minio" ) var ctx = context.Background() var client, err = minio_driver.NewMinioDriver(ctx) - func TestMinioDriver_PutRowAndGetRow(t *testing.T) { err = client.PutRow(ctx, []byte("bar"), []byte("abcdefghijklmnoopqrstuvwxyz"), "SegmentA", 1) assert.Nil(t, err) @@ -30,7 +30,7 @@ func TestMinioDriver_PutRowAndGetRow(t *testing.T) { assert.Equal(t, "testkeybarorbar_1", string(object)) } -func TestMinioDriver_DeleteRow(t *testing.T){ +func TestMinioDriver_DeleteRow(t *testing.T) { err = client.DeleteRow(ctx, []byte("bar"), 5) assert.Nil(t, err) object, _ := client.GetRow(ctx, []byte("bar"), 6) @@ -63,7 +63,7 @@ func TestMinioDriver_GetSegments(t *testing.T) { } } -func TestMinioDriver_PutRowsAndGetRows(t *testing.T){ +func TestMinioDriver_PutRowsAndGetRows(t *testing.T) { keys := [][]byte{[]byte("foo"), []byte("bar")} values := [][]byte{[]byte("The key is foo!"), []byte("The key is bar!")} segments := []string{"segmentA", "segmentB"} @@ -77,7 +77,7 @@ func TestMinioDriver_PutRowsAndGetRows(t *testing.T){ assert.Equal(t, "The key is bar!", string(objects[1])) } -func TestMinioDriver_DeleteRows(t *testing.T){ +func TestMinioDriver_DeleteRows(t *testing.T) { keys := [][]byte{[]byte("foo"), []byte("bar")} timestamps := []uint64{3, 3} err := client.DeleteRows(ctx, keys, timestamps) @@ -92,11 +92,11 @@ func TestMinioDriver_DeleteRows(t *testing.T){ func TestMinioDriver_PutLogAndGetLog(t *testing.T) { err = client.PutLog(ctx, []byte("insert"), []byte("This is insert log!"), 1, 11) assert.Nil(t, err) - err = client.PutLog(ctx, []byte("delete"), []byte("This is delete log!"), 2, 10) + err = client.PutLog(ctx, []byte("delete"), []byte("This is delete log!"), 2, 10) assert.Nil(t, err) - err = client.PutLog(ctx, []byte("update"), []byte("This is update log!"), 3, 9) + err = client.PutLog(ctx, []byte("update"), []byte("This is update log!"), 3, 9) assert.Nil(t, err) - err = client.PutLog(ctx, []byte("select"), []byte("This is select log!"), 4, 8) + err = client.PutLog(ctx, []byte("select"), []byte("This is select log!"), 4, 8) assert.Nil(t, err) channels := []int{5, 8, 9, 10, 11, 12, 13} @@ -119,7 +119,7 @@ func TestMinioDriver_Segment(t *testing.T) { assert.Nil(t, err) } -func TestMinioDriver_SegmentDL(t *testing.T){ +func TestMinioDriver_SegmentDL(t *testing.T) { err := client.PutSegmentDL(ctx, "segmentB", []byte("This is segmentB's delete log!")) assert.Nil(t, err) @@ -129,4 +129,4 @@ func TestMinioDriver_SegmentDL(t *testing.T){ err = client.DeleteSegmentDL(ctx, "segmentB") assert.Nil(t, err) -} \ No newline at end of file +} diff --git a/internal/storage/internal/tikv/codec/codec.go b/internal/storage/internal/tikv/codec/codec.go index d69f0eb298..ca09296097 100644 --- a/internal/storage/internal/tikv/codec/codec.go +++ b/internal/storage/internal/tikv/codec/codec.go @@ -3,6 +3,7 @@ package codec import ( "encoding/binary" "errors" + "github.com/tikv/client-go/codec" ) diff --git a/internal/storage/internal/tikv/tikv_store.go b/internal/storage/internal/tikv/tikv_store.go index 104a527894..ab3d1547d2 100644 --- a/internal/storage/internal/tikv/tikv_store.go +++ b/internal/storage/internal/tikv/tikv_store.go @@ -3,13 +3,14 @@ package tikv_driver import ( "context" "errors" + "strconv" + "strings" + + "github.com/tikv/client-go/config" + "github.com/tikv/client-go/rawkv" "github.com/zilliztech/milvus-distributed/internal/conf" . "github.com/zilliztech/milvus-distributed/internal/storage/internal/tikv/codec" . "github.com/zilliztech/milvus-distributed/internal/storage/type" - "github.com/tikv/client-go/config" - "github.com/tikv/client-go/rawkv" - "strconv" - "strings" ) func keyAddOne(key Key) Key { @@ -366,7 +367,7 @@ func (s *TikvStore) GetSegments(ctx context.Context, key Key, timestamp Timestam if err != nil { panic("must no error") } - if ts <= timestamp && segment != string(DeleteMark){ + if ts <= timestamp && segment != string(DeleteMark) { segmentsSet[segment] = true } } diff --git a/internal/storage/internal/tikv/tikv_test.go b/internal/storage/internal/tikv/tikv_test.go index a3e6e1f100..3139bf6e2b 100644 --- a/internal/storage/internal/tikv/tikv_test.go +++ b/internal/storage/internal/tikv/tikv_test.go @@ -4,14 +4,15 @@ import ( "bytes" "context" "fmt" - . "github.com/zilliztech/milvus-distributed/internal/storage/internal/tikv/codec" - . "github.com/zilliztech/milvus-distributed/internal/storage/type" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" "math" "os" "sort" "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + . "github.com/zilliztech/milvus-distributed/internal/storage/internal/tikv/codec" + . "github.com/zilliztech/milvus-distributed/internal/storage/type" ) //var store TikvStore diff --git a/internal/storage/storage.go b/internal/storage/storage.go index 4c79647370..0c3cb788c7 100644 --- a/internal/storage/storage.go +++ b/internal/storage/storage.go @@ -3,16 +3,17 @@ package storage import ( "context" "errors" + S3Driver "github.com/zilliztech/milvus-distributed/internal/storage/internal/S3" minIODriver "github.com/zilliztech/milvus-distributed/internal/storage/internal/minio" tikvDriver "github.com/zilliztech/milvus-distributed/internal/storage/internal/tikv" - "github.com/zilliztech/milvus-distributed/internal/storage/type" + storagetype "github.com/zilliztech/milvus-distributed/internal/storage/type" ) func NewStore(ctx context.Context, driver storagetype.DriverType) (storagetype.Store, error) { var err error var store storagetype.Store - switch driver{ + switch driver { case storagetype.TIKVDriver: store, err = tikvDriver.NewTikvStore(ctx) if err != nil { @@ -27,7 +28,7 @@ func NewStore(ctx context.Context, driver storagetype.DriverType) (storagetype.S } return store, nil case storagetype.S3DRIVER: - store , err = S3Driver.NewS3Driver(ctx) + store, err = S3Driver.NewS3Driver(ctx) if err != nil { //panic(err.Error()) return nil, err @@ -35,4 +36,4 @@ func NewStore(ctx context.Context, driver storagetype.DriverType) (storagetype.S return store, nil } return nil, errors.New("unsupported driver") -} \ No newline at end of file +} diff --git a/internal/storage/type/storagetype.go b/internal/storage/type/storagetype.go index 55afd08211..7295f3e1eb 100644 --- a/internal/storage/type/storagetype.go +++ b/internal/storage/type/storagetype.go @@ -14,7 +14,7 @@ type SegmentDL = []byte const ( MinIODriver DriverType = "MinIO" TIKVDriver DriverType = "TIKV" - S3DRIVER DriverType = "S3" + S3DRIVER DriverType = "S3" ) /* diff --git a/internal/timesync/timesync.go b/internal/timesync/timesync.go index 6b1a9b95e6..f143bc58f6 100644 --- a/internal/timesync/timesync.go +++ b/internal/timesync/timesync.go @@ -12,10 +12,10 @@ import ( "time" "github.com/apache/pulsar-client-go/pulsar" + "github.com/golang/protobuf/proto" "github.com/zilliztech/milvus-distributed/internal/conf" "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" pb "github.com/zilliztech/milvus-distributed/internal/proto/message" - "github.com/golang/protobuf/proto" ) type InsertLog struct { diff --git a/internal/timesync/timesync_test.go b/internal/timesync/timesync_test.go index fcb1fb4096..302f59c7c9 100644 --- a/internal/timesync/timesync_test.go +++ b/internal/timesync/timesync_test.go @@ -8,9 +8,9 @@ import ( "time" "github.com/apache/pulsar-client-go/pulsar" + "github.com/golang/protobuf/proto" "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" pb "github.com/zilliztech/milvus-distributed/internal/proto/message" - "github.com/golang/protobuf/proto" ) const ( @@ -52,15 +52,15 @@ func TestAlignTimeSync(t *testing.T) { } ts := []*internalpb.TimeSyncMsg{ { - PeerId: 1, + PeerId: 1, Timestamp: toTimestamp(5), }, { - PeerId: 3, + PeerId: 3, Timestamp: toTimestamp(15), }, { - PeerId: 2, + PeerId: 2, Timestamp: toTimestamp(20), }, } @@ -83,15 +83,15 @@ func TestAlignTimeSync2(t *testing.T) { } ts := []*internalpb.TimeSyncMsg{ { - PeerId: 1, + PeerId: 1, Timestamp: toTimestamp(5), }, { - PeerId: 3, + PeerId: 3, Timestamp: toTimestamp(150), }, { - PeerId: 2, + PeerId: 2, Timestamp: toTimestamp(20), }, } @@ -112,23 +112,23 @@ func TestAlignTimeSync3(t *testing.T) { } ts := []*internalpb.TimeSyncMsg{ { - PeerId: 1, + PeerId: 1, Timestamp: toTimestamp(5), }, { - PeerId: 1, + PeerId: 1, Timestamp: toTimestamp(5), }, { - PeerId: 1, + PeerId: 1, Timestamp: toTimestamp(5), }, { - PeerId: 3, + PeerId: 3, Timestamp: toTimestamp(15), }, { - PeerId: 2, + PeerId: 2, Timestamp: toTimestamp(20), }, } @@ -150,15 +150,15 @@ func TestAlignTimeSync4(t *testing.T) { } ts := []*internalpb.TimeSyncMsg{ { - PeerId: 1, + PeerId: 1, Timestamp: toTimestamp(15), }, { - PeerId: 1, + PeerId: 1, Timestamp: toTimestamp(25), }, { - PeerId: 1, + PeerId: 1, Timestamp: toTimestamp(35), }, } @@ -181,23 +181,23 @@ func TestAlignTimeSync5(t *testing.T) { } ts := []*internalpb.TimeSyncMsg{ { - PeerId: 1, + PeerId: 1, Timestamp: toTimestamp(5), }, { - PeerId: 1, + PeerId: 1, Timestamp: toTimestamp(5), }, { - PeerId: 1, + PeerId: 1, Timestamp: toTimestamp(5), }, { - PeerId: 3, + PeerId: 3, Timestamp: toTimestamp(15), }, { - PeerId: 3, + PeerId: 3, Timestamp: toTimestamp(20), }, } diff --git a/internal/util/etcdutil/etcdutil.go b/internal/util/etcdutil/etcdutil.go index 45a2f6d045..3ec2e88815 100644 --- a/internal/util/etcdutil/etcdutil.go +++ b/internal/util/etcdutil/etcdutil.go @@ -1,16 +1,3 @@ -// Copyright 2016 TiKV Project Authors. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - package etcdutil import ( @@ -19,9 +6,9 @@ import ( "net/http" "time" - "github.com/zilliztech/milvus-distributed/internal/errors" "github.com/gogo/protobuf/proto" "github.com/pingcap/log" + "github.com/zilliztech/milvus-distributed/internal/errors" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/etcdserver" "go.etcd.io/etcd/pkg/types" diff --git a/internal/util/etcdutil/etcdutil_test.go b/internal/util/etcdutil/etcdutil_test.go index f5b942d226..64b69c54c9 100644 --- a/internal/util/etcdutil/etcdutil_test.go +++ b/internal/util/etcdutil/etcdutil_test.go @@ -1,16 +1,3 @@ -// Copyright 2016 TiKV Project Authors. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - package etcdutil import ( diff --git a/internal/util/flowgraph/flow_graph.go b/internal/util/flowgraph/flow_graph.go index b775d64682..57c87ab4b0 100644 --- a/internal/util/flowgraph/flow_graph.go +++ b/internal/util/flowgraph/flow_graph.go @@ -2,9 +2,10 @@ package flowgraph import ( "context" + "sync" + "github.com/zilliztech/milvus-distributed/internal/errors" "github.com/zilliztech/milvus-distributed/internal/util/typeutil" - "sync" ) type Timestamp = typeutil.Timestamp @@ -24,8 +25,8 @@ type TimeTickedFlowGraph struct { func (fg *TimeTickedFlowGraph) AddNode(node *Node) { nodeName := (*node).Name() nodeCtx := nodeCtx{ - node: node, - inputChannels: make([]chan *Msg, 0), + node: node, + inputChannels: make([]chan *Msg, 0), downstreamInputChanIdx: make(map[string]int), } fg.nodeCtx[nodeName] = &nodeCtx diff --git a/internal/util/tsoutil/tso.go b/internal/util/tsoutil/tso.go index 9e977a52c7..625c16635d 100644 --- a/internal/util/tsoutil/tso.go +++ b/internal/util/tsoutil/tso.go @@ -1,16 +1,3 @@ -// Copyright 2019 TiKV Project Authors. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - package tsoutil import ( @@ -22,7 +9,7 @@ const ( logicalBits = (1 << physicalShiftBits) - 1 ) -func ComposeTS(physical, logical int64) uint64{ +func ComposeTS(physical, logical int64) uint64 { return uint64((physical << physicalShiftBits) + logical) } @@ -33,4 +20,3 @@ func ParseTS(ts uint64) (time.Time, uint64) { physicalTime := time.Unix(int64(physical/1000), int64(physical)%1000*time.Millisecond.Nanoseconds()) return physicalTime, logical } - diff --git a/internal/util/typeutil/convension.go b/internal/util/typeutil/convension.go index 674df398af..a444f79b81 100644 --- a/internal/util/typeutil/convension.go +++ b/internal/util/typeutil/convension.go @@ -1,20 +1,8 @@ -// Copyright 2016 TiKV Project Authors. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - package typeutil import ( "encoding/binary" + "github.com/zilliztech/milvus-distributed/internal/errors" ) diff --git a/internal/util/typeutil/hash.go b/internal/util/typeutil/hash.go index 9d67410448..0973274c2f 100644 --- a/internal/util/typeutil/hash.go +++ b/internal/util/typeutil/hash.go @@ -2,8 +2,9 @@ package typeutil import ( "encoding/binary" - "github.com/spaolacci/murmur3" "unsafe" + + "github.com/spaolacci/murmur3" ) func Hash32Bytes(b []byte) (uint32, error) { @@ -20,7 +21,6 @@ func Hash32Uint64(v uint64) (uint32, error) { return Hash32Bytes(b) } - func Hash32Int64(v int64) (uint32, error) { return Hash32Uint64(uint64(v)) } diff --git a/internal/util/typeutil/hash_test.go b/internal/util/typeutil/hash_test.go index c7f642e503..035659131c 100644 --- a/internal/util/typeutil/hash_test.go +++ b/internal/util/typeutil/hash_test.go @@ -1,8 +1,10 @@ package typeutil + import ( - "github.com/stretchr/testify/assert" "testing" "unsafe" + + "github.com/stretchr/testify/assert" ) func TestUint64(t *testing.T) { @@ -27,4 +29,3 @@ func TestHash32_Uint64(t *testing.T) { t.Log(h2) assert.Equal(t, h, h2) } - diff --git a/internal/util/typeutil/time.go b/internal/util/typeutil/time.go index f12c87f04e..f81d3d66da 100644 --- a/internal/util/typeutil/time.go +++ b/internal/util/typeutil/time.go @@ -1,16 +1,3 @@ -// Copyright 2016 TiKV Project Authors. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - package typeutil import "time" diff --git a/internal/util/typeutil/type.go b/internal/util/typeutil/type.go index 14e2fe188a..e4fd306089 100644 --- a/internal/util/typeutil/type.go +++ b/internal/util/typeutil/type.go @@ -1,5 +1,4 @@ package typeutil - type Timestamp = uint64 -type Id = int64 \ No newline at end of file +type Id = int64 diff --git a/internal/writer/writer.go b/internal/writer/writer.go index f882c905c3..cd07207669 100644 --- a/internal/writer/writer.go +++ b/internal/writer/writer.go @@ -5,16 +5,17 @@ import ( "encoding/binary" "encoding/json" "fmt" - "github.com/zilliztech/milvus-distributed/internal/conf" - msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message" - "github.com/zilliztech/milvus-distributed/internal/storage" - storagetype "github.com/zilliztech/milvus-distributed/internal/storage/type" - "github.com/zilliztech/milvus-distributed/internal/msgclient" "log" "os" "strconv" "sync" "time" + + "github.com/zilliztech/milvus-distributed/internal/conf" + "github.com/zilliztech/milvus-distributed/internal/msgclient" + msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message" + "github.com/zilliztech/milvus-distributed/internal/storage" + storagetype "github.com/zilliztech/milvus-distributed/internal/storage/type" ) type SegmentIdInfo struct { diff --git a/test/insert_test.go b/test/insert_test.go index ee036bb8d2..6fda8808ed 100644 --- a/test/insert_test.go +++ b/test/insert_test.go @@ -2,10 +2,11 @@ package test import ( "context" - msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message" - "github.com/zilliztech/milvus-distributed/internal/writer" "sync" "testing" + + msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message" + "github.com/zilliztech/milvus-distributed/internal/writer" ) func GetInsertMsg(collectionName string, partitionTag string, entityId int64) *msgpb.InsertOrDeleteMsg { @@ -13,7 +14,7 @@ func GetInsertMsg(collectionName string, partitionTag string, entityId int64) *m CollectionName: collectionName, PartitionTag: partitionTag, SegmentId: int64(entityId / 100), - Uid: int64(entityId), + Uid: int64(entityId), Timestamp: uint64(entityId), ClientId: 0, } @@ -22,7 +23,7 @@ func GetInsertMsg(collectionName string, partitionTag string, entityId int64) *m func GetDeleteMsg(collectionName string, entityId int64) *msgpb.InsertOrDeleteMsg { return &msgpb.InsertOrDeleteMsg{ CollectionName: collectionName, - Uid: entityId, + Uid: entityId, Timestamp: uint64(entityId + 100), } } diff --git a/test/key2seg_test.go b/test/key2seg_test.go index 3a48d2ba9e..6b19bf188e 100644 --- a/test/key2seg_test.go +++ b/test/key2seg_test.go @@ -2,12 +2,13 @@ package test import ( "context" - "github.com/apache/pulsar-client-go/pulsar" - msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message" - "github.com/gogo/protobuf/proto" - "github.com/stretchr/testify/assert" "log" "testing" + + "github.com/apache/pulsar-client-go/pulsar" + "github.com/gogo/protobuf/proto" + "github.com/stretchr/testify/assert" + msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message" ) func TestKey2Seg(t *testing.T) {