Add timestamp allocator

Signed-off-by: zhenshan.cao <zhenshan.cao@zilliz.com>
This commit is contained in:
zhenshan.cao 2020-11-03 14:53:36 +08:00 committed by yefu.chen
parent d7ef21c644
commit e962b57fbb
97 changed files with 1602 additions and 1668 deletions

View File

@ -12,7 +12,6 @@ import (
"syscall" "syscall"
) )
func main() { func main() {
var yamlFile string var yamlFile string

View File

@ -12,7 +12,7 @@ import (
func main() { func main() {
ctx, _ := context.WithCancel(context.Background()) ctx, _ := context.WithCancel(context.Background())
var yamlFile string var yamlFile string
flag.StringVar(&yamlFile, "yaml", "", "yaml file") flag.StringVar(&yamlFile, "yaml", "", "yaml file")
flag.Parse() flag.Parse()
// flag.Usage() // flag.Usage()
@ -25,4 +25,3 @@ func main() {
pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10) pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
reader.StartQueryNode(ctx, pulsarAddr) reader.StartQueryNode(ctx, pulsarAddr)
} }

View File

@ -5,9 +5,9 @@ import (
"crypto/md5" "crypto/md5"
"flag" "flag"
"fmt" "fmt"
"github.com/pivotal-golang/bytefmt"
"github.com/zilliztech/milvus-distributed/internal/storage" "github.com/zilliztech/milvus-distributed/internal/storage"
"github.com/zilliztech/milvus-distributed/internal/storage/type" "github.com/zilliztech/milvus-distributed/internal/storage/type"
"github.com/pivotal-golang/bytefmt"
"log" "log"
"math/rand" "math/rand"
"os" "os"
@ -31,14 +31,13 @@ var logFile *os.File
var store storagetype.Store var store storagetype.Store
var wg sync.WaitGroup var wg sync.WaitGroup
func runSet() { func runSet() {
for time.Now().Before(endTime) { for time.Now().Before(endTime) {
num := atomic.AddInt32(&keyNum, 1) num := atomic.AddInt32(&keyNum, 1)
key := []byte(fmt.Sprint("key", num)) key := []byte(fmt.Sprint("key", num))
for ver := 1; ver <= numVersion; ver++ { for ver := 1; ver <= numVersion; ver++ {
atomic.AddInt32(&counter, 1) 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 { if err != nil {
log.Fatalf("Error setting key %s, %s", key, err.Error()) log.Fatalf("Error setting key %s, %s", key, err.Error())
//atomic.AddInt32(&setCount, -1) //atomic.AddInt32(&setCount, -1)
@ -61,7 +60,7 @@ func runBatchSet() {
} }
for ver := 1; ver <= numVersion; ver++ { for ver := 1; ver <= numVersion; ver++ {
atomic.AddInt32(&counter, 1) 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 { if err != nil {
log.Fatalf("Error setting batch keys %s %s", keys, err.Error()) log.Fatalf("Error setting batch keys %s %s", keys, err.Error())
//atomic.AddInt32(&batchSetCount, -1) //atomic.AddInt32(&batchSetCount, -1)
@ -72,7 +71,6 @@ func runBatchSet() {
wg.Done() wg.Done()
} }
func runGet() { func runGet() {
for time.Now().Before(endTime) { for time.Now().Before(endTime) {
num := atomic.AddInt32(&counter, 1) num := atomic.AddInt32(&counter, 1)
@ -105,8 +103,8 @@ func runBatchGet() {
start := end - int32(batchOpSize) start := end - int32(batchOpSize)
keys := totalKeys[start:end] keys := totalKeys[start:end]
versions := make([]uint64, batchOpSize) versions := make([]uint64, batchOpSize)
for i, _ := range versions{ for i, _ := range versions {
versions[i]= uint64(numVersion) versions[i] = uint64(numVersion)
} }
atomic.AddInt32(&counter, 1) atomic.AddInt32(&counter, 1)
_, err := store.GetRows(context.Background(), keys, versions) _, err := store.GetRows(context.Background(), keys, versions)
@ -153,8 +151,8 @@ func runBatchDelete() {
keys := totalKeys[start:end] keys := totalKeys[start:end]
atomic.AddInt32(&counter, 1) atomic.AddInt32(&counter, 1)
versions := make([]uint64, batchOpSize) versions := make([]uint64, batchOpSize)
for i, _ := range versions{ for i, _ := range versions {
versions[i]= uint64(numVersion) versions[i] = uint64(numVersion)
} }
err := store.DeleteRows(context.Background(), keys, versions) err := store.DeleteRows(context.Background(), keys, versions)
if err != nil { if err != nil {
@ -239,7 +237,7 @@ func main() {
setTime := setFinish.Sub(startTime).Seconds() setTime := setFinish.Sub(startTime).Seconds()
bps := float64(uint64(counter)*valueSize*uint64(batchOpSize)) / setTime 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", 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 // Record all test keys
//totalKeyCount = keyNum //totalKeyCount = keyNum

View File

@ -5,18 +5,18 @@ import (
"flag" "flag"
"fmt" "fmt"
"github.com/zilliztech/milvus-distributed/internal/conf" "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/msgclient"
"github.com/zilliztech/milvus-distributed/internal/storage"
"github.com/zilliztech/milvus-distributed/internal/writer" "github.com/zilliztech/milvus-distributed/internal/writer"
"log" "log"
"strconv" "strconv"
) )
func main() { func main() {
var yamlFile string var yamlFile string
flag.StringVar(&yamlFile, "yaml", "", "yaml file") flag.StringVar(&yamlFile, "yaml", "", "yaml file")
flag.Parse() flag.Parse()
// flag.Usage() // flag.Usage()
fmt.Println("yaml file: ", yamlFile) fmt.Println("yaml file: ", yamlFile)
conf.LoadConfig(yamlFile) conf.LoadConfig(yamlFile)
@ -63,7 +63,7 @@ func main() {
if wn.MsgCounter.InsertCounter/CountInsertMsgBaseline != BaselineCounter { if wn.MsgCounter.InsertCounter/CountInsertMsgBaseline != BaselineCounter {
wn.WriteWriterLog() wn.WriteWriterLog()
BaselineCounter = wn.MsgCounter.InsertCounter/CountInsertMsgBaseline BaselineCounter = wn.MsgCounter.InsertCounter / CountInsertMsgBaseline
} }
if msgLength > 0 { if msgLength > 0 {

View File

@ -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"))
}

View File

@ -1,35 +1,79 @@
package allocator package allocator
import ( import (
"github.com/zilliztech/milvus-distributed/internal/util/typeutil" "context"
"fmt"
"log"
"time"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
) )
type IdAllocator struct { 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 { func (ta *IdAllocator) processFunc(req request) {
return nil idRequest := req.(*idRequest)
idRequest.id = 1
fmt.Println("process Id")
} }
func (allocator *IdAllocator) Start() error{ func (ta *IdAllocator) AllocOne() (int64, error) {
return nil ret, _, err := ta.Alloc(1)
} if err != nil {
func (allocator *IdAllocator) Close() error{ return 0, err
return nil }
return ret, nil
} }
func (allocator *IdAllocator) AllocOne() typeutil.Id { func (ta *IdAllocator) Alloc(count uint32) (int64, int64, error) {
return 1 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{}
}

View File

@ -1,32 +1,98 @@
package allocator package allocator
import ( 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{ const (
return nil tsCountPerRPC = 2 << 18 * 10
defaultUpdateInterval = 1000 * time.Millisecond
)
type TimestampAllocator struct {
Allocator
lastTsBegin uint64
lastTsEnd uint64
} }
func (allocator *TimestampAllocator) Close() error{ func NewTimestampAllocator(ctx context.Context) (*TimestampAllocator, error) {
return nil 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){ func (ta *TimestampAllocator) syncTs() {
ret, err := allocator.Alloc(1) fmt.Println("sync TS")
if err != nil{ ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
return typeutil.ZeroTimestamp, err 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 return ret[0], nil
} }
func (allocator *TimestampAllocator) Alloc(count uint32) ([]typeutil.Timestamp, error){ func (ta *TimestampAllocator) Alloc(count uint32) ([]Timestamp, error) {
// to do lock and accuire more by grpc request //req := tsoReqPool.Get().(*tsoRequest)
return make([]typeutil.Timestamp, count), nil 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{ if !req.IsValid() {
return &TimestampAllocator{} 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
} }

View File

@ -8,4 +8,3 @@ import (
func TestMain(m *testing.M) { func TestMain(m *testing.M) {
fmt.Printf("Result: %v\n", Config) fmt.Printf("Result: %v\n", Config)
} }

View File

@ -1,4 +1,3 @@
package errors package errors
import ( import (

View File

@ -2,11 +2,12 @@ package kv
import ( import (
"context" "context"
"github.com/zilliztech/milvus-distributed/internal/errors"
"go.etcd.io/etcd/clientv3"
"log" "log"
"path" "path"
"time" "time"
"github.com/zilliztech/milvus-distributed/internal/errors"
"go.etcd.io/etcd/clientv3"
) )
const ( const (

View File

@ -1,10 +1,11 @@
package kv package kv
import ( import (
"github.com/stretchr/testify/assert"
"go.etcd.io/etcd/clientv3"
"path" "path"
"testing" "testing"
"github.com/stretchr/testify/assert"
"go.etcd.io/etcd/clientv3"
) )
func TestEtcdKV_Load(t *testing.T) { func TestEtcdKV_Load(t *testing.T) {

View File

@ -3,29 +3,29 @@ package collection
import ( import (
"time" "time"
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"github.com/gogo/protobuf/proto" "github.com/gogo/protobuf/proto"
jsoniter "github.com/json-iterator/go" 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 var json = jsoniter.ConfigCompatibleWithStandardLibrary
type Collection struct { type Collection struct {
ID int64 `json:"id"` ID int64 `json:"id"`
Name string `json:"name"` Name string `json:"name"`
CreateTime uint64 `json:"creat_time"` CreateTime uint64 `json:"creat_time"`
Schema []FieldMeta `json:"schema"` Schema []FieldMeta `json:"schema"`
// ExtraSchema []FieldMeta `json:"extra_schema"` // ExtraSchema []FieldMeta `json:"extra_schema"`
SegmentIDs []int64 `json:"segment_ids"` SegmentIDs []int64 `json:"segment_ids"`
PartitionTags []string `json:"partition_tags"` PartitionTags []string `json:"partition_tags"`
GrpcMarshalString string `json:"grpc_marshal_string"` GrpcMarshalString string `json:"grpc_marshal_string"`
} }
type FieldMeta struct { type FieldMeta struct {
FieldName string `json:"field_name"` FieldName string `json:"field_name"`
Type schemapb.DataType `json:"type"` Type schemapb.DataType `json:"type"`
DIM int64 `json:"dimension"` DIM int64 `json:"dimension"`
} }
func GrpcMarshal(c *Collection) *Collection { func GrpcMarshal(c *Collection) *Collection {
@ -38,8 +38,8 @@ func GrpcMarshal(c *Collection) *Collection {
schemaSlice := []*schemapb.FieldSchema{} schemaSlice := []*schemapb.FieldSchema{}
for _, v := range c.Schema { for _, v := range c.Schema {
newpbMeta := &schemapb.FieldSchema{ newpbMeta := &schemapb.FieldSchema{
Name: v.FieldName, Name: v.FieldName,
DataType: schemapb.DataType(v.Type), //czs_tag DataType: schemapb.DataType(v.Type), //czs_tag
} }
schemaSlice = append(schemaSlice, newpbMeta) schemaSlice = append(schemaSlice, newpbMeta)
} }

View File

@ -3,8 +3,9 @@ package collection
import ( import (
"testing" "testing"
"time" "time"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
) )
var ( var (

View File

@ -3,13 +3,14 @@ package master
import ( import (
"encoding/json" "encoding/json"
"errors" "errors"
"log"
"strconv"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb" "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb" "github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb" "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb" "github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb" "github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
"log"
"strconv"
) )
const collectionMetaPrefix = "collection/" const collectionMetaPrefix = "collection/"

View File

@ -5,21 +5,20 @@ import (
"strconv" "strconv"
"time" "time"
"github.com/zilliztech/milvus-distributed/internal/conf" "github.com/zilliztech/milvus-distributed/internal/master/id"
"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"
)
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) { func CollectionController(ch chan *schemapb.CollectionSchema, kvbase kv.Base, errch chan error) {
for collectionMeta := range ch { for collectionMeta := range ch {
sID := IdAllocator.AllocOne() sID, _ := id.AllocOne()
cID := IdAllocator.AllocOne() cID, _ := id.AllocOne()
s2ID := IdAllocator.AllocOne() s2ID, _ := id.AllocOne()
fieldMetas := []*schemapb.FieldSchema{} fieldMetas := []*schemapb.FieldSchema{}
if collectionMeta.Fields != nil { if collectionMeta.Fields != nil {
fieldMetas = collectionMeta.Fields 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 { func WriteCollection2Datastore(collectionMeta *schemapb.CollectionSchema, kvbase kv.Base) error {
sID := IdAllocator.AllocOne() sID, _ := id.AllocOne()
cID := IdAllocator.AllocOne() cID, _ := id.AllocOne()
fieldMetas := []*schemapb.FieldSchema{} fieldMetas := []*schemapb.FieldSchema{}
if collectionMeta.Fields != nil { if collectionMeta.Fields != nil {
fieldMetas = collectionMeta.Fields fieldMetas = collectionMeta.Fields

View File

@ -5,13 +5,14 @@ import (
"strconv" "strconv"
"time" "time"
"github.com/zilliztech/milvus-distributed/internal/conf" "github.com/zilliztech/milvus-distributed/internal/master/id"
"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/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 { func ComputeCloseTime(ss internalpb.SegmentStatistics, kvbase kv.Base) error {
if int(ss.MemorySize) > int(conf.Config.Master.SegmentThreshole*0.8) { 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) kvbase.Save("segment/"+strconv.Itoa(int(ss.SegmentId)), updateData)
//create new segment //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)) 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) newSegData, err := segment.Segment2JSON(*&newSeg)
if err != nil { if err != nil {
@ -63,4 +64,3 @@ func ComputeCloseTime(ss internalpb.SegmentStatistics, kvbase kv.Base) error {
} }
return nil return nil
} }

View File

@ -28,7 +28,7 @@ func TestComputeClosetTime(t *testing.T) {
var news internalpb.SegmentStatistics var news internalpb.SegmentStatistics
for i := 0; i < 10; i++ { for i := 0; i < 10; i++ {
news = internalpb.SegmentStatistics{ news = internalpb.SegmentStatistics{
SegmentId: int64(6875940398055133887), SegmentId: int64(6875940398055133887),
MemorySize: int64(i * 1000), MemorySize: int64(i * 1000),
} }
ComputeCloseTime(news, kvbase) ComputeCloseTime(news, kvbase)

View File

@ -2,17 +2,13 @@ package master
import ( import (
"context" "context"
"time"
"github.com/zilliztech/milvus-distributed/internal/errors" "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/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb" "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb" "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 const slowThreshold = 5 * time.Millisecond
@ -340,38 +336,43 @@ func (s *Master) ShowPartitions(ctx context.Context, in *internalpb.ShowPartitio
return t.(*showPartitionTask).stringListResponse, nil return t.(*showPartitionTask).stringListResponse, nil
} }
//----------------------------------------Internal GRPC Service-------------------------------- //----------------------------------------Internal GRPC Service--------------------------------
// Tso implements gRPC PDServer. func (s *Master) AllocTimestamp(ctx context.Context, request *internalpb.TsoRequest) (*internalpb.TsoResponse, error) {
func (s *Master) Tso(stream masterpb.Master_TsoServer) error { count := request.GetCount()
for { ts, err := s.tsoAllocator.GenerateTSO(count)
request, err := stream.Recv()
if err == io.EOF {
return nil
}
if err != nil {
return errors.WithStack(err)
}
start := time.Now()
count := request.GetCount() if err != nil {
ts, err := s.tsoAllocator.GenerateTSO(count) return &internalpb.TsoResponse{
if err != nil { Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR},
return status.Errorf(codes.Unknown, err.Error()) }, err
}
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)
}
} }
}
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
}

View File

@ -17,16 +17,15 @@ import (
"github.com/zilliztech/milvus-distributed/internal/master/tso" "github.com/zilliztech/milvus-distributed/internal/master/tso"
) )
// GlobalTSOAllocator is the global single point TSO allocator. // GlobalTSOAllocator is the global single point TSO allocator.
type GlobalIdAllocator struct { type GlobalIdAllocator struct {
allocator tso.Allocator allocator tso.Allocator
} }
func NewGlobalIdAllocator() *GlobalIdAllocator { //func getID
return &GlobalIdAllocator{
allocator: tso.NewGlobalTSOAllocator("idTimestamp"), var allocator GlobalIdAllocator = GlobalIdAllocator{
} allocator: tso.NewGlobalTSOAllocator("idTimestamp"),
} }
// Initialize will initialize the created global TSO allocator. // 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. // GenerateTSO is used to generate a given number of TSOs.
// Make sure you have initialized the TSO allocator before calling. // Make sure you have initialized the TSO allocator before calling.
func (gia *GlobalIdAllocator) Generate(count uint32) (int64, int64, error) { func (gia *GlobalIdAllocator) Alloc(count uint32) (int64, int64, error) {
timestamp, err:= gia.allocator.GenerateTSO(count) timestamp, err := gia.allocator.GenerateTSO(count)
if err != nil{ if err != nil {
return 0, 0, err return 0, 0, err
} }
idStart := int64(timestamp) idStart := int64(timestamp)
@ -46,3 +45,19 @@ func (gia *GlobalIdAllocator) Generate(count uint32) (int64, int64, error) {
return idStart, idEnd, nil 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)
}

View File

@ -3,6 +3,14 @@ package master
import ( import (
"context" "context"
"fmt" "fmt"
"log"
"math/rand"
"net"
"strconv"
"sync"
"sync/atomic"
"time"
"github.com/apache/pulsar-client-go/pulsar" "github.com/apache/pulsar-client-go/pulsar"
"github.com/golang/protobuf/proto" "github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/conf" "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/internalpb"
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb" "github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
"google.golang.org/grpc" "google.golang.org/grpc"
"log"
"math/rand"
"net"
"strconv"
"sync"
"sync/atomic"
"time"
"github.com/zilliztech/milvus-distributed/internal/master/tso" "github.com/zilliztech/milvus-distributed/internal/master/tso"
"go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/clientv3"

View File

@ -1,12 +1,13 @@
package master package master
import ( import (
"strconv"
"sync"
"github.com/golang/protobuf/proto" "github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/errors" "github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/kv" "github.com/zilliztech/milvus-distributed/internal/kv"
pb "github.com/zilliztech/milvus-distributed/internal/proto/etcdpb" pb "github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
"strconv"
"sync"
) )
type metaTable struct { type metaTable struct {

View File

@ -1,12 +1,13 @@
package master package master
import ( import (
"testing"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/kv" "github.com/zilliztech/milvus-distributed/internal/kv"
pb "github.com/zilliztech/milvus-distributed/internal/proto/etcdpb" pb "github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb" "github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/clientv3"
"testing"
) )
func TestMetaTable_DeletePartition(t *testing.T) { func TestMetaTable_DeletePartition(t *testing.T) {

View File

@ -3,11 +3,12 @@ package master
import ( import (
"encoding/json" "encoding/json"
"errors" "errors"
"log"
"strconv"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb" "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb" "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb" "github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
"log"
"strconv"
) )
const partitionMetaPrefix = "partition/" const partitionMetaPrefix = "partition/"
@ -195,7 +196,7 @@ func (t *describePartitionTask) Execute() error {
partitionName := t.req.PartitionName partitionName := t.req.PartitionName
description := servicepb.PartitionDescription { description := servicepb.PartitionDescription{
Status: &commonpb.Status{ Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS, ErrorCode: commonpb.ErrorCode_SUCCESS,
}, },
@ -237,7 +238,7 @@ func (t *showPartitionTask) Execute() error {
} }
} }
stringListResponse := servicepb.StringListResponse { stringListResponse := servicepb.StringListResponse{
Status: &commonpb.Status{ Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS, ErrorCode: commonpb.ErrorCode_SUCCESS,
}, },

View File

@ -9,15 +9,15 @@ import (
var json = jsoniter.ConfigCompatibleWithStandardLibrary var json = jsoniter.ConfigCompatibleWithStandardLibrary
type Segment struct { type Segment struct {
SegmentID int64 `json:"segment_id"` SegmentID int64 `json:"segment_id"`
CollectionID int64 `json:"collection_id"` CollectionID int64 `json:"collection_id"`
PartitionTag string `json:"partition_tag"` PartitionTag string `json:"partition_tag"`
ChannelStart int `json:"channel_start"` ChannelStart int `json:"channel_start"`
ChannelEnd int `json:"channel_end"` ChannelEnd int `json:"channel_end"`
OpenTimeStamp uint64 `json:"open_timestamp"` OpenTimeStamp uint64 `json:"open_timestamp"`
CloseTimeStamp uint64 `json:"close_timestamp"` CloseTimeStamp uint64 `json:"close_timestamp"`
CollectionName string `json:"collection_name"` CollectionName string `json:"collection_name"`
Rows int64 `json:"rows"` 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 { func NewSegment(id int64, collectioID int64, cName string, ptag string, chStart int, chEnd int, openTime time.Time, closeTime time.Time) Segment {

View File

@ -2,6 +2,7 @@ package master
import ( import (
"context" "context"
"github.com/zilliztech/milvus-distributed/internal/errors" "github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/kv" "github.com/zilliztech/milvus-distributed/internal/kv"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb" "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"

View File

@ -2,13 +2,14 @@ package timesync
import ( import (
"context" "context"
"github.com/zilliztech/milvus-distributed/internal/conf"
"log" "log"
"sort" "sort"
"strconv" "strconv"
"sync" "sync"
"time" "time"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/apache/pulsar-client-go/pulsar" "github.com/apache/pulsar-client-go/pulsar"
"github.com/golang/protobuf/proto" "github.com/golang/protobuf/proto"
pb "github.com/zilliztech/milvus-distributed/internal/proto/message" pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
@ -29,7 +30,7 @@ type TimeTickReader struct {
ctx context.Context ctx context.Context
} }
func (r *TimeTickReader) Start(){ func (r *TimeTickReader) Start() {
go r.readTimeTick() go r.readTimeTick()
go r.timeSync() go r.timeSync()
} }
@ -113,7 +114,6 @@ func (r *TimeTickReader) sendEOFMsg(ctx context.Context, msg *pulsar.ProducerMes
wg.Done() wg.Done()
} }
func TimeTickService() { func TimeTickService() {
timeTickTopic := "timeTick" timeTickTopic := "timeTick"
timeTickSubName := "master" timeTickSubName := "master"

View File

@ -14,13 +14,14 @@
package tso package tso
import ( import (
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
"go.etcd.io/etcd/clientv3"
"strconv" "strconv"
"sync/atomic" "sync/atomic"
"time" "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/pingcap/log"
"github.com/zilliztech/milvus-distributed/internal/errors" "github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil" "github.com/zilliztech/milvus-distributed/internal/util/typeutil"
@ -62,14 +63,14 @@ func NewGlobalTSOAllocator(key string) Allocator {
DialTimeout: 5 * time.Second, DialTimeout: 5 * time.Second,
}) })
var saveInterval time.Duration = 3 *time.Second var saveInterval time.Duration = 3 * time.Second
return &GlobalTSOAllocator{ return &GlobalTSOAllocator{
timestampOracle: &timestampOracle{ timestampOracle: &timestampOracle{
client: client, client: client,
rootPath: conf.Config.Etcd.Rootpath, rootPath: conf.Config.Etcd.Rootpath,
saveInterval: saveInterval, saveInterval: saveInterval,
maxResetTSGap: func() time.Duration { return 3 *time.Second}, maxResetTSGap: func() time.Duration { return 3 * time.Second },
key: key, key: key,
}, },
} }
} }

View File

@ -14,13 +14,14 @@
package tso package tso
import ( import (
"go.uber.org/zap"
"log" "log"
"path" "path"
"sync/atomic" "sync/atomic"
"time" "time"
"unsafe" "unsafe"
"go.uber.org/zap"
"github.com/zilliztech/milvus-distributed/internal/errors" "github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/util/etcdutil" "github.com/zilliztech/milvus-distributed/internal/util/etcdutil"
"github.com/zilliztech/milvus-distributed/internal/util/tsoutil" "github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
@ -49,7 +50,7 @@ type atomicObject struct {
type timestampOracle struct { type timestampOracle struct {
client *clientv3.Client client *clientv3.Client
rootPath string rootPath string
key string key string
// TODO: remove saveInterval // TODO: remove saveInterval
saveInterval time.Duration saveInterval time.Duration
maxResetTSGap func() 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, // save timestamp, if lastTs is 0, we think the timestamp doesn't exist, so create it,
// otherwise, update it. // otherwise, update it.
func (t *timestampOracle) saveTimestamp( ts time.Time) error { func (t *timestampOracle) saveTimestamp(ts time.Time) error {
key := t.getTimestampPath() key := t.getTimestampPath()
data := typeutil.Uint64ToBytes(uint64(ts.UnixNano())) data := typeutil.Uint64ToBytes(uint64(ts.UnixNano()))
err := errors.New("") err := errors.New("")
println("%v,%v",key, data) println("%v,%v", key, data)
//resp, err := leadership.LeaderTxn(). //resp, err := leadership.LeaderTxn().
// Then(clientv3.OpPut(key, string(data))). // Then(clientv3.OpPut(key, string(data))).
// Commit() // Commit()
@ -127,7 +128,7 @@ func (t *timestampOracle) SyncTimestamp() error {
} }
// ResetUserTimestamp update the physical part with specified tso. // 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() { //if !leadership.Check() {
// return errors.New("Setup timestamp failed, lease expired") // return errors.New("Setup timestamp failed, lease expired")
//} //}
@ -145,7 +146,7 @@ func (t *timestampOracle) ResetUserTimestamp( tso uint64) error {
} }
save := next.Add(t.saveInterval) save := next.Add(t.saveInterval)
if err := t.saveTimestamp( save); err != nil { if err := t.saveTimestamp(save); err != nil {
return err return err
} }
update := &atomicObject{ update := &atomicObject{
@ -194,7 +195,7 @@ func (t *timestampOracle) UpdateTimestamp() error {
// The time window needs to be updated and saved to etcd. // The time window needs to be updated and saved to etcd.
if typeutil.SubTimeByWallClock(t.lastSavedTime.Load().(time.Time), next) <= updateTimestampGuard { if typeutil.SubTimeByWallClock(t.lastSavedTime.Load().(time.Time), next) <= updateTimestampGuard {
save := next.Add(t.saveInterval) save := next.Add(t.saveInterval)
if err := t.saveTimestamp( save); err != nil { if err := t.saveTimestamp(save); err != nil {
return err return err
} }
} }

View File

@ -7,11 +7,11 @@ import (
"strconv" "strconv"
"github.com/apache/pulsar-client-go/pulsar" "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/conf"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb" "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message" msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/zilliztech/milvus-distributed/internal/timesync" "github.com/zilliztech/milvus-distributed/internal/timesync"
"github.com/golang/protobuf/proto"
) )
type MessageType int type MessageType int

View File

@ -59,7 +59,7 @@ func (im *InsertMarshaler) Marshal(input *TsMsg) ([]byte, commonPb.Status) {
func (im *InsertMarshaler) Unmarshal(input []byte) (*TsMsg, commonPb.Status) { func (im *InsertMarshaler) Unmarshal(input []byte) (*TsMsg, commonPb.Status) {
insertRequest := internalPb.InsertRequest{} insertRequest := internalPb.InsertRequest{}
err := proto.Unmarshal(input, &insertRequest) err := proto.Unmarshal(input, &insertRequest)
insertTask := InsertTask{InsertRequest:insertRequest} insertTask := InsertTask{InsertRequest: insertRequest}
if err != nil { if err != nil {
return nil, commonPb.Status{ErrorCode: commonPb.ErrorCode_UNEXPECTED_ERROR} 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) { func (dm *DeleteMarshaler) Unmarshal(input []byte) (*TsMsg, commonPb.Status) {
deleteRequest := internalPb.DeleteRequest{} deleteRequest := internalPb.DeleteRequest{}
err := proto.Unmarshal(input, &deleteRequest) err := proto.Unmarshal(input, &deleteRequest)
deleteTask := DeleteTask{DeleteRequest:deleteRequest} deleteTask := DeleteTask{DeleteRequest: deleteRequest}
if err != nil { if err != nil {
return nil, commonPb.Status{ErrorCode: commonPb.ErrorCode_UNEXPECTED_ERROR} 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) { func (tm *TimeSyncMarshaler) Unmarshal(input []byte) (*TsMsg, commonPb.Status) {
timeSyncMsg := internalPb.TimeSyncMsg{} timeSyncMsg := internalPb.TimeSyncMsg{}
err := proto.Unmarshal(input, &timeSyncMsg) err := proto.Unmarshal(input, &timeSyncMsg)
timeSyncTask := TimeSyncTask{TimeSyncMsg:timeSyncMsg} timeSyncTask := TimeSyncTask{TimeSyncMsg: timeSyncMsg}
if err != nil { if err != nil {
return nil, commonPb.Status{ErrorCode: commonPb.ErrorCode_UNEXPECTED_ERROR} return nil, commonPb.Status{ErrorCode: commonPb.ErrorCode_UNEXPECTED_ERROR}
} }

View File

@ -2,17 +2,17 @@ package msgstream
import ( import (
"context" "context"
"github.com/apache/pulsar-client-go/pulsar"
commonPb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"log" "log"
"sync" "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 { type MsgPack struct {
BeginTs Timestamp BeginTs typeutil.Timestamp
EndTs Timestamp EndTs typeutil.Timestamp
Msgs []*TsMsg Msgs []*TsMsg
} }
@ -29,12 +29,12 @@ type MsgStream interface {
} }
type PulsarMsgStream struct { type PulsarMsgStream struct {
client *pulsar.Client client *pulsar.Client
producers []*pulsar.Producer producers []*pulsar.Producer
consumers []*pulsar.Consumer consumers []*pulsar.Consumer
repackFunc RepackFunc // return a map from produceChannel idx to *MsgPack repackFunc RepackFunc // return a map from produceChannel idx to *MsgPack
receiveBuf chan *MsgPack receiveBuf chan *MsgPack
msgMarshaler *TsMsgMarshaler msgMarshaler *TsMsgMarshaler
msgUnmarshaler *TsMsgMarshaler msgUnmarshaler *TsMsgMarshaler
@ -86,7 +86,7 @@ func (ms *PulsarMsgStream) SetRepackFunc(repackFunc RepackFunc) {
ms.repackFunc = repackFunc ms.repackFunc = repackFunc
} }
func (ms *PulsarMsgStream) Start(){ func (ms *PulsarMsgStream) Start() {
go ms.bufMsgPackToChannel() go ms.bufMsgPackToChannel()
} }
@ -110,10 +110,9 @@ func (ms *PulsarMsgStream) InitMsgPackBuf(msgPackBufSize int64) {
ms.receiveBuf = make(chan *MsgPack, msgPackBufSize) ms.receiveBuf = make(chan *MsgPack, msgPackBufSize)
} }
func (ms *PulsarMsgStream) Produce(msgPack *MsgPack) commonPb.Status { func (ms *PulsarMsgStream) Produce(msgPack *MsgPack) commonPb.Status {
tsMsgs := msgPack.Msgs tsMsgs := msgPack.Msgs
if len(tsMsgs) <=0 { if len(tsMsgs) <= 0 {
log.Println("receive empty msgPack") log.Println("receive empty msgPack")
return commonPb.Status{ErrorCode: commonPb.ErrorCode_SUCCESS} return commonPb.Status{ErrorCode: commonPb.ErrorCode_SUCCESS}
} }
@ -215,17 +214,17 @@ type PulsarTtMsgStream struct {
inputBuf []*TsMsg inputBuf []*TsMsg
unsolvedBuf []*TsMsg unsolvedBuf []*TsMsg
msgPacks []*MsgPack msgPacks []*MsgPack
lastTimeStamp Timestamp lastTimeStamp typeutil.Timestamp
} }
func (ms *PulsarTtMsgStream) Start(){ func (ms *PulsarTtMsgStream) Start() {
go ms.bufMsgPackToChannel() go ms.bufMsgPackToChannel()
} }
func (ms *PulsarTtMsgStream) bufMsgPackToChannel() { func (ms *PulsarTtMsgStream) bufMsgPackToChannel() {
wg := sync.WaitGroup{} wg := sync.WaitGroup{}
wg.Add(len(ms.consumers)) wg.Add(len(ms.consumers))
eofMsgTimeStamp := make(map[int]Timestamp) eofMsgTimeStamp := make(map[int]typeutil.Timestamp)
mu := sync.Mutex{} mu := sync.Mutex{}
for i := 0; i < len(ms.consumers); i++ { for i := 0; i < len(ms.consumers); i++ {
go ms.findTimeTick(context.Background(), i, eofMsgTimeStamp, &wg, &mu) go ms.findTimeTick(context.Background(), i, eofMsgTimeStamp, &wg, &mu)
@ -259,7 +258,7 @@ func (ms *PulsarTtMsgStream) bufMsgPackToChannel() {
func (ms *PulsarTtMsgStream) findTimeTick(ctx context.Context, func (ms *PulsarTtMsgStream) findTimeTick(ctx context.Context,
channelIndex int, channelIndex int,
eofMsgMap map[int]Timestamp, eofMsgMap map[int]typeutil.Timestamp,
wg *sync.WaitGroup, wg *sync.WaitGroup,
mu *sync.Mutex) { mu *sync.Mutex) {
for { for {
@ -289,8 +288,8 @@ func (ms *PulsarTtMsgStream) findTimeTick(ctx context.Context,
} }
} }
func checkTimeTickMsg(msg map[int]Timestamp) (Timestamp, bool) { func checkTimeTickMsg(msg map[int]typeutil.Timestamp) (typeutil.Timestamp, bool) {
checkMap := make(map[Timestamp]int) checkMap := make(map[typeutil.Timestamp]int)
for _, v := range msg { for _, v := range msg {
checkMap[v] += 1 checkMap[v] += 1
} }

View File

@ -2,9 +2,10 @@ package msgstream
import ( import (
"fmt" "fmt"
"testing"
commonPb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb" commonPb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"testing"
) )
func repackFunc(msgs []*TsMsg, hashKeys [][]int32) map[int32]*MsgPack { 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}, Timestamps: []uint64{1},
} }
insertMsg := InsertTask{ insertMsg := InsertTask{
HashValues: []int32{hashValue}, HashValues: []int32{hashValue},
InsertRequest: insertRequest, InsertRequest: insertRequest,
} }
tsMsg = insertMsg tsMsg = insertMsg
@ -49,11 +50,11 @@ func getTsMsg(msgType MsgType, reqId int64, hashValue int32) *TsMsg {
CollectionName: "Collection", CollectionName: "Collection",
ChannelId: 1, ChannelId: 1,
ProxyId: 1, ProxyId: 1,
Timestamps: []uint64{1}, Timestamps: []uint64{1},
PrimaryKeys: []int64{1}, PrimaryKeys: []int64{1},
} }
deleteMsg := DeleteTask{ deleteMsg := DeleteTask{
HashValues: []int32{hashValue}, HashValues: []int32{hashValue},
DeleteRequest: deleteRequest, DeleteRequest: deleteRequest,
} }
tsMsg = deleteMsg tsMsg = deleteMsg
@ -66,7 +67,7 @@ func getTsMsg(msgType MsgType, reqId int64, hashValue int32) *TsMsg {
ResultChannelId: 1, ResultChannelId: 1,
} }
searchMsg := SearchTask{ searchMsg := SearchTask{
HashValues: []int32{hashValue}, HashValues: []int32{hashValue},
SearchRequest: searchRequest, SearchRequest: searchRequest,
} }
tsMsg = searchMsg tsMsg = searchMsg
@ -80,7 +81,7 @@ func getTsMsg(msgType MsgType, reqId int64, hashValue int32) *TsMsg {
ResultChannelId: 1, ResultChannelId: 1,
} }
searchResultMsg := SearchResultTask{ searchResultMsg := SearchResultTask{
HashValues: []int32{hashValue}, HashValues: []int32{hashValue},
SearchResult: searchResult, SearchResult: searchResult,
} }
tsMsg = searchResultMsg tsMsg = searchResultMsg
@ -90,7 +91,7 @@ func getTsMsg(msgType MsgType, reqId int64, hashValue int32) *TsMsg {
Timestamp: 1, Timestamp: 1,
} }
timeSyncMsg := TimeSyncTask{ timeSyncMsg := TimeSyncTask{
HashValues: []int32{hashValue}, HashValues: []int32{hashValue},
TimeSyncMsg: timeSyncResult, TimeSyncMsg: timeSyncResult,
} }
tsMsg = timeSyncMsg tsMsg = timeSyncMsg
@ -106,7 +107,7 @@ func getTsMsg(msgType MsgType, reqId int64, hashValue int32) *TsMsg {
Timestamps: []uint64{1}, Timestamps: []uint64{1},
} }
insertMsg := InsertTask{ insertMsg := InsertTask{
HashValues: []int32{hashValue}, HashValues: []int32{hashValue},
InsertRequest: insertRequest, InsertRequest: insertRequest,
} }
tsMsg = insertMsg tsMsg = insertMsg

View File

@ -2,8 +2,9 @@ package msgstream
import ( import (
"fmt" "fmt"
internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"testing" "testing"
internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
) )
func TestNewStream_Insert(t *testing.T) { func TestNewStream_Insert(t *testing.T) {
@ -216,7 +217,7 @@ func TestNewStream_Insert_TimeTick(t *testing.T) {
Timestamps: []uint64{1}, Timestamps: []uint64{1},
} }
insertMsg := InsertTask{ insertMsg := InsertTask{
HashValues: []int32{2}, HashValues: []int32{2},
InsertRequest: insertRequest, InsertRequest: insertRequest,
} }
var tsMsg TsMsg = insertMsg var tsMsg TsMsg = insertMsg
@ -244,8 +245,8 @@ func TestNewStream_Insert_TimeTick(t *testing.T) {
fmt.Println("msg type: ", (*v).Type(), ", msg value: ", *v) fmt.Println("msg type: ", (*v).Type(), ", msg value: ", *v)
} }
} }
if receiveCount + 1 >= len(msgPack.Msgs) { if receiveCount+1 >= len(msgPack.Msgs) {
break break
} }
} }
} }

View File

@ -2,15 +2,16 @@ package msgstream
import ( import (
internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb" internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
. "github.com/zilliztech/milvus-distributed/internal/util/typeutil"
) )
type MsgType uint32 type MsgType uint32
const ( const (
kInsert MsgType = 400 kInsert MsgType = 400
kDelete MsgType = 401 kDelete MsgType = 401
kSearch MsgType = 500 kSearch MsgType = 500
kSearchResult MsgType = 1000 kSearchResult MsgType = 1000
kSegmentStatics MsgType = 1100 kSegmentStatics MsgType = 1100
kTimeTick MsgType = 1200 kTimeTick MsgType = 1200
@ -166,11 +167,11 @@ func (srt SearchResultTask) SetTs(ts Timestamp) {
srt.Timestamp = uint64(ts) srt.Timestamp = uint64(ts)
} }
func (srt SearchResultTask) BeginTs() Timestamp { func (srt SearchResultTask) BeginTs() Timestamp {
return Timestamp(srt.Timestamp) return Timestamp(srt.Timestamp)
} }
func (srt SearchResultTask) EndTs() Timestamp { func (srt SearchResultTask) EndTs() Timestamp {
return Timestamp(srt.Timestamp) return Timestamp(srt.Timestamp)
} }
@ -196,7 +197,7 @@ func (tst TimeSyncTask) BeginTs() Timestamp {
return Timestamp(tst.Timestamp) return Timestamp(tst.Timestamp)
} }
func (tst TimeSyncTask) EndTs() Timestamp { func (tst TimeSyncTask) EndTs() Timestamp {
return Timestamp(tst.Timestamp) return Timestamp(tst.Timestamp)
} }
@ -208,7 +209,6 @@ func (tst TimeSyncTask) HashKeys() []int32 {
return tst.HashValues return tst.HashValues
} }
///////////////////////////////////////////Key2Seg////////////////////////////////////////// ///////////////////////////////////////////Key2Seg//////////////////////////////////////////
//type Key2SegTask struct { //type Key2SegTask struct {
// internalPb.Key2SegMsg // internalPb.Key2SegMsg

View File

@ -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 { message TsoRequest {
int64 peer_id = 1; int64 peer_id = 1;
PeerRole role = 2; PeerRole role = 2;

View File

@ -122,6 +122,116 @@ func (PeerRole) EnumDescriptor() ([]byte, []int) {
return fileDescriptor_7eb37f6b80b23116, []int{1} 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 { type TsoRequest struct {
PeerId int64 `protobuf:"varint,1,opt,name=peer_id,json=peerId,proto3" json:"peer_id,omitempty"` PeerId int64 `protobuf:"varint,1,opt,name=peer_id,json=peerId,proto3" json:"peer_id,omitempty"`
Role PeerRole `protobuf:"varint,2,opt,name=role,proto3,enum=milvus.proto.internal.PeerRole" json:"role,omitempty"` 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 (m *TsoRequest) String() string { return proto.CompactTextString(m) }
func (*TsoRequest) ProtoMessage() {} func (*TsoRequest) ProtoMessage() {}
func (*TsoRequest) Descriptor() ([]byte, []int) { func (*TsoRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_7eb37f6b80b23116, []int{0} return fileDescriptor_7eb37f6b80b23116, []int{2}
} }
func (m *TsoRequest) XXX_Unmarshal(b []byte) error { 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 (m *TsoResponse) String() string { return proto.CompactTextString(m) }
func (*TsoResponse) ProtoMessage() {} func (*TsoResponse) ProtoMessage() {}
func (*TsoResponse) Descriptor() ([]byte, []int) { func (*TsoResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_7eb37f6b80b23116, []int{1} return fileDescriptor_7eb37f6b80b23116, []int{3}
} }
func (m *TsoResponse) XXX_Unmarshal(b []byte) error { 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 (m *CreateCollectionRequest) String() string { return proto.CompactTextString(m) }
func (*CreateCollectionRequest) ProtoMessage() {} func (*CreateCollectionRequest) ProtoMessage() {}
func (*CreateCollectionRequest) Descriptor() ([]byte, []int) { func (*CreateCollectionRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_7eb37f6b80b23116, []int{2} return fileDescriptor_7eb37f6b80b23116, []int{4}
} }
func (m *CreateCollectionRequest) XXX_Unmarshal(b []byte) error { 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 (m *DropCollectionRequest) String() string { return proto.CompactTextString(m) }
func (*DropCollectionRequest) ProtoMessage() {} func (*DropCollectionRequest) ProtoMessage() {}
func (*DropCollectionRequest) Descriptor() ([]byte, []int) { func (*DropCollectionRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_7eb37f6b80b23116, []int{3} return fileDescriptor_7eb37f6b80b23116, []int{5}
} }
func (m *DropCollectionRequest) XXX_Unmarshal(b []byte) error { 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 (m *HasCollectionRequest) String() string { return proto.CompactTextString(m) }
func (*HasCollectionRequest) ProtoMessage() {} func (*HasCollectionRequest) ProtoMessage() {}
func (*HasCollectionRequest) Descriptor() ([]byte, []int) { func (*HasCollectionRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_7eb37f6b80b23116, []int{4} return fileDescriptor_7eb37f6b80b23116, []int{6}
} }
func (m *HasCollectionRequest) XXX_Unmarshal(b []byte) error { 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 (m *DescribeCollectionRequest) String() string { return proto.CompactTextString(m) }
func (*DescribeCollectionRequest) ProtoMessage() {} func (*DescribeCollectionRequest) ProtoMessage() {}
func (*DescribeCollectionRequest) Descriptor() ([]byte, []int) { func (*DescribeCollectionRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_7eb37f6b80b23116, []int{5} return fileDescriptor_7eb37f6b80b23116, []int{7}
} }
func (m *DescribeCollectionRequest) XXX_Unmarshal(b []byte) error { 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 (m *ShowCollectionRequest) String() string { return proto.CompactTextString(m) }
func (*ShowCollectionRequest) ProtoMessage() {} func (*ShowCollectionRequest) ProtoMessage() {}
func (*ShowCollectionRequest) Descriptor() ([]byte, []int) { func (*ShowCollectionRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_7eb37f6b80b23116, []int{6} return fileDescriptor_7eb37f6b80b23116, []int{8}
} }
func (m *ShowCollectionRequest) XXX_Unmarshal(b []byte) error { 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 (m *CreatePartitionRequest) String() string { return proto.CompactTextString(m) }
func (*CreatePartitionRequest) ProtoMessage() {} func (*CreatePartitionRequest) ProtoMessage() {}
func (*CreatePartitionRequest) Descriptor() ([]byte, []int) { func (*CreatePartitionRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_7eb37f6b80b23116, []int{7} return fileDescriptor_7eb37f6b80b23116, []int{9}
} }
func (m *CreatePartitionRequest) XXX_Unmarshal(b []byte) error { 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 (m *DropPartitionRequest) String() string { return proto.CompactTextString(m) }
func (*DropPartitionRequest) ProtoMessage() {} func (*DropPartitionRequest) ProtoMessage() {}
func (*DropPartitionRequest) Descriptor() ([]byte, []int) { func (*DropPartitionRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_7eb37f6b80b23116, []int{8} return fileDescriptor_7eb37f6b80b23116, []int{10}
} }
func (m *DropPartitionRequest) XXX_Unmarshal(b []byte) error { 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 (m *HasPartitionRequest) String() string { return proto.CompactTextString(m) }
func (*HasPartitionRequest) ProtoMessage() {} func (*HasPartitionRequest) ProtoMessage() {}
func (*HasPartitionRequest) Descriptor() ([]byte, []int) { func (*HasPartitionRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_7eb37f6b80b23116, []int{9} return fileDescriptor_7eb37f6b80b23116, []int{11}
} }
func (m *HasPartitionRequest) XXX_Unmarshal(b []byte) error { 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 (m *DescribePartitionRequest) String() string { return proto.CompactTextString(m) }
func (*DescribePartitionRequest) ProtoMessage() {} func (*DescribePartitionRequest) ProtoMessage() {}
func (*DescribePartitionRequest) Descriptor() ([]byte, []int) { func (*DescribePartitionRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_7eb37f6b80b23116, []int{10} return fileDescriptor_7eb37f6b80b23116, []int{12}
} }
func (m *DescribePartitionRequest) XXX_Unmarshal(b []byte) error { 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 (m *ShowPartitionRequest) String() string { return proto.CompactTextString(m) }
func (*ShowPartitionRequest) ProtoMessage() {} func (*ShowPartitionRequest) ProtoMessage() {}
func (*ShowPartitionRequest) Descriptor() ([]byte, []int) { func (*ShowPartitionRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_7eb37f6b80b23116, []int{11} return fileDescriptor_7eb37f6b80b23116, []int{13}
} }
func (m *ShowPartitionRequest) XXX_Unmarshal(b []byte) error { 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 (m *InsertRequest) String() string { return proto.CompactTextString(m) }
func (*InsertRequest) ProtoMessage() {} func (*InsertRequest) ProtoMessage() {}
func (*InsertRequest) Descriptor() ([]byte, []int) { func (*InsertRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_7eb37f6b80b23116, []int{12} return fileDescriptor_7eb37f6b80b23116, []int{14}
} }
func (m *InsertRequest) XXX_Unmarshal(b []byte) error { 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 (m *DeleteRequest) String() string { return proto.CompactTextString(m) }
func (*DeleteRequest) ProtoMessage() {} func (*DeleteRequest) ProtoMessage() {}
func (*DeleteRequest) Descriptor() ([]byte, []int) { func (*DeleteRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_7eb37f6b80b23116, []int{13} return fileDescriptor_7eb37f6b80b23116, []int{15}
} }
func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { 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 (m *SearchRequest) String() string { return proto.CompactTextString(m) }
func (*SearchRequest) ProtoMessage() {} func (*SearchRequest) ProtoMessage() {}
func (*SearchRequest) Descriptor() ([]byte, []int) { func (*SearchRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_7eb37f6b80b23116, []int{14} return fileDescriptor_7eb37f6b80b23116, []int{16}
} }
func (m *SearchRequest) XXX_Unmarshal(b []byte) error { 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 (m *SearchResult) String() string { return proto.CompactTextString(m) }
func (*SearchResult) ProtoMessage() {} func (*SearchResult) ProtoMessage() {}
func (*SearchResult) Descriptor() ([]byte, []int) { func (*SearchResult) Descriptor() ([]byte, []int) {
return fileDescriptor_7eb37f6b80b23116, []int{15} return fileDescriptor_7eb37f6b80b23116, []int{17}
} }
func (m *SearchResult) XXX_Unmarshal(b []byte) error { 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 (m *TimeSyncMsg) String() string { return proto.CompactTextString(m) }
func (*TimeSyncMsg) ProtoMessage() {} func (*TimeSyncMsg) ProtoMessage() {}
func (*TimeSyncMsg) Descriptor() ([]byte, []int) { func (*TimeSyncMsg) Descriptor() ([]byte, []int) {
return fileDescriptor_7eb37f6b80b23116, []int{16} return fileDescriptor_7eb37f6b80b23116, []int{18}
} }
func (m *TimeSyncMsg) XXX_Unmarshal(b []byte) error { 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 (m *Key2Seg) String() string { return proto.CompactTextString(m) }
func (*Key2Seg) ProtoMessage() {} func (*Key2Seg) ProtoMessage() {}
func (*Key2Seg) Descriptor() ([]byte, []int) { func (*Key2Seg) Descriptor() ([]byte, []int) {
return fileDescriptor_7eb37f6b80b23116, []int{17} return fileDescriptor_7eb37f6b80b23116, []int{19}
} }
func (m *Key2Seg) XXX_Unmarshal(b []byte) error { 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 (m *Key2SegMsg) String() string { return proto.CompactTextString(m) }
func (*Key2SegMsg) ProtoMessage() {} func (*Key2SegMsg) ProtoMessage() {}
func (*Key2SegMsg) Descriptor() ([]byte, []int) { func (*Key2SegMsg) Descriptor() ([]byte, []int) {
return fileDescriptor_7eb37f6b80b23116, []int{18} return fileDescriptor_7eb37f6b80b23116, []int{20}
} }
func (m *Key2SegMsg) XXX_Unmarshal(b []byte) error { 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 (m *SegmentStatistics) String() string { return proto.CompactTextString(m) }
func (*SegmentStatistics) ProtoMessage() {} func (*SegmentStatistics) ProtoMessage() {}
func (*SegmentStatistics) Descriptor() ([]byte, []int) { func (*SegmentStatistics) Descriptor() ([]byte, []int) {
return fileDescriptor_7eb37f6b80b23116, []int{19} return fileDescriptor_7eb37f6b80b23116, []int{21}
} }
func (m *SegmentStatistics) XXX_Unmarshal(b []byte) error { func (m *SegmentStatistics) XXX_Unmarshal(b []byte) error {
@ -1521,6 +1631,8 @@ func (m *SegmentStatistics) GetNumRows() int64 {
func init() { func init() {
proto.RegisterEnum("milvus.proto.internal.ReqType", ReqType_name, ReqType_value) proto.RegisterEnum("milvus.proto.internal.ReqType", ReqType_name, ReqType_value)
proto.RegisterEnum("milvus.proto.internal.PeerRole", PeerRole_name, PeerRole_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((*TsoRequest)(nil), "milvus.proto.internal.TsoRequest")
proto.RegisterType((*TsoResponse)(nil), "milvus.proto.internal.TsoResponse") proto.RegisterType((*TsoResponse)(nil), "milvus.proto.internal.TsoResponse")
proto.RegisterType((*CreateCollectionRequest)(nil), "milvus.proto.internal.CreateCollectionRequest") proto.RegisterType((*CreateCollectionRequest)(nil), "milvus.proto.internal.CreateCollectionRequest")
@ -1546,77 +1658,79 @@ func init() {
func init() { proto.RegisterFile("internal_msg.proto", fileDescriptor_7eb37f6b80b23116) } func init() { proto.RegisterFile("internal_msg.proto", fileDescriptor_7eb37f6b80b23116) }
var fileDescriptor_7eb37f6b80b23116 = []byte{ var fileDescriptor_7eb37f6b80b23116 = []byte{
// 1147 bytes of a gzipped FileDescriptorProto // 1171 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x58, 0x49, 0x6f, 0x23, 0x45, 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x58, 0x4b, 0x6f, 0x1b, 0x55,
0x14, 0x9e, 0xf6, 0xee, 0xe7, 0xd8, 0xe9, 0x54, 0x62, 0xe2, 0x19, 0x60, 0xc6, 0xf4, 0x20, 0x11, 0x14, 0xee, 0xf8, 0x19, 0x1f, 0xc7, 0xce, 0xe4, 0x26, 0x21, 0x6e, 0x81, 0xd6, 0x4c, 0x91, 0x88,
0x8d, 0x84, 0x23, 0x3c, 0x1c, 0x98, 0xeb, 0xc4, 0x87, 0x98, 0x51, 0xa2, 0xa8, 0x1d, 0x81, 0x84, 0x2a, 0xe1, 0x08, 0x97, 0x05, 0xdd, 0xb6, 0x5e, 0xd4, 0x54, 0xad, 0xaa, 0x71, 0x04, 0x12, 0x12,
0x84, 0x5a, 0xed, 0xee, 0x87, 0x5d, 0xea, 0x35, 0x55, 0xe5, 0x04, 0xe7, 0x0f, 0x70, 0x05, 0x71, 0x1a, 0x8d, 0x67, 0x0e, 0xf6, 0xd5, 0x3c, 0xee, 0xe4, 0xde, 0xeb, 0x04, 0xe7, 0x0f, 0xb0, 0x05,
0xe4, 0xc6, 0x99, 0x03, 0x3f, 0x83, 0xed, 0xce, 0x9f, 0x00, 0xc1, 0x48, 0xa0, 0xb9, 0xa2, 0xaa, 0xb1, 0x64, 0xc7, 0x9a, 0x05, 0x3f, 0x83, 0xd7, 0x9e, 0x3f, 0x01, 0x82, 0x4a, 0xa0, 0x6e, 0xd1,
0x6e, 0x2f, 0xed, 0x2c, 0x2c, 0x62, 0xa4, 0x48, 0xb9, 0xf5, 0x7b, 0xb5, 0xbd, 0xef, 0x7b, 0x8b, 0xbd, 0x33, 0x7e, 0x8c, 0xf3, 0xe0, 0x59, 0x14, 0x29, 0xbb, 0x39, 0xe7, 0x3e, 0xce, 0xf9, 0xbe,
0xdf, 0x33, 0x10, 0x1a, 0x0a, 0x64, 0xa1, 0xed, 0x5b, 0x01, 0x1f, 0x75, 0x62, 0x16, 0x89, 0x88, 0xf3, 0xf0, 0x3d, 0x06, 0x42, 0x63, 0x89, 0x3c, 0x76, 0x43, 0x27, 0x12, 0xa3, 0x4e, 0xc2, 0x99,
0x34, 0x03, 0xea, 0x9f, 0x4e, 0x78, 0x22, 0x75, 0x66, 0x1b, 0xee, 0xad, 0x39, 0x51, 0x10, 0x44, 0x64, 0x64, 0x27, 0xa2, 0xe1, 0xd1, 0x44, 0xa4, 0x52, 0x67, 0xb6, 0xe1, 0xc6, 0xba, 0xc7, 0xa2,
0x61, 0xa2, 0xbe, 0xb7, 0xc1, 0x91, 0x9d, 0x52, 0x07, 0x17, 0xe7, 0x8c, 0x18, 0xe0, 0x98, 0x47, 0x88, 0xc5, 0xa9, 0xfa, 0xc6, 0xa6, 0x40, 0x7e, 0x44, 0x3d, 0x5c, 0x9c, 0xb3, 0x18, 0xd4, 0xfa,
0x26, 0x9e, 0x4c, 0x90, 0x0b, 0xb2, 0x0d, 0xe5, 0x18, 0x91, 0x59, 0xd4, 0x6d, 0x69, 0x6d, 0x6d, 0xbe, 0x8d, 0x87, 0x13, 0x14, 0x92, 0xec, 0x42, 0x35, 0x41, 0xe4, 0x0e, 0xf5, 0x5b, 0x46, 0xdb,
0x27, 0x6f, 0x96, 0xa4, 0xd8, 0x77, 0xc9, 0x63, 0x28, 0xb0, 0xc8, 0xc7, 0x56, 0xae, 0xad, 0xed, 0xd8, 0x2b, 0xda, 0x15, 0x25, 0xf6, 0x7d, 0x72, 0x17, 0x4a, 0x9c, 0x85, 0xd8, 0x2a, 0xb4, 0x8d,
0x34, 0xba, 0x0f, 0x3a, 0x97, 0xbe, 0xd6, 0x39, 0x42, 0x64, 0x66, 0xe4, 0xa3, 0xa9, 0x36, 0x93, 0xbd, 0x66, 0xf7, 0x56, 0xe7, 0x4c, 0x63, 0x9d, 0xa7, 0x88, 0xdc, 0x66, 0x21, 0xda, 0x7a, 0x33,
0x2d, 0x28, 0x3a, 0xd1, 0x24, 0x14, 0xad, 0x7c, 0x5b, 0xdb, 0xa9, 0x9b, 0x89, 0x60, 0x9c, 0x42, 0xd9, 0x86, 0xb2, 0xc7, 0x26, 0xb1, 0x6c, 0x15, 0xdb, 0xc6, 0x5e, 0xc3, 0x4e, 0x05, 0x6b, 0x04,
0x4d, 0xbd, 0xc8, 0xe3, 0x28, 0xe4, 0x48, 0x1e, 0x43, 0x89, 0x0b, 0x5b, 0x4c, 0xb8, 0x7a, 0xb1, 0xa0, 0x0c, 0x8a, 0x84, 0xc5, 0x02, 0xc9, 0x5d, 0xa8, 0x08, 0xe9, 0xca, 0x89, 0xd0, 0x06, 0xeb,
0xd6, 0x7d, 0x35, 0x7b, 0x77, 0x6a, 0xff, 0x40, 0x6d, 0x31, 0xd3, 0xad, 0xe4, 0x35, 0xa8, 0x0a, 0xdd, 0x97, 0xf3, 0x57, 0x67, 0xde, 0x0f, 0xf4, 0x16, 0x3b, 0xdb, 0x4a, 0x9a, 0x50, 0xa0, 0xbe,
0x1a, 0x20, 0x17, 0x76, 0x10, 0x2b, 0x9b, 0x0a, 0xe6, 0x42, 0x71, 0xc5, 0xbb, 0x3f, 0x6b, 0xb0, 0xf6, 0xa5, 0x68, 0x17, 0xa8, 0x7f, 0x8e, 0xa1, 0x04, 0xe0, 0x40, 0xb0, 0xff, 0x13, 0xda, 0x11,
0xbd, 0xc7, 0xd0, 0x16, 0xb8, 0x17, 0xf9, 0x3e, 0x3a, 0x82, 0x46, 0xe1, 0x0c, 0xf7, 0x13, 0xa8, 0xd4, 0xb5, 0xc5, 0x7f, 0x83, 0xed, 0x15, 0xa8, 0x49, 0x1a, 0xa1, 0x90, 0x6e, 0x94, 0x68, 0x9f,
0x30, 0x3c, 0xb1, 0xc4, 0x34, 0x46, 0x65, 0x46, 0xa3, 0x7b, 0xff, 0x0a, 0x88, 0x26, 0x9e, 0x1c, 0x4a, 0xf6, 0x42, 0x71, 0x8e, 0xdd, 0x1f, 0x0d, 0xd8, 0x7d, 0xc0, 0xd1, 0x95, 0xf8, 0x80, 0x85,
0x4f, 0x63, 0x34, 0xcb, 0x2c, 0xf9, 0x20, 0x4d, 0x28, 0xc9, 0xa3, 0xd4, 0x55, 0x76, 0xe4, 0xcd, 0x21, 0x7a, 0x92, 0xb2, 0x78, 0x86, 0xfb, 0x1e, 0xac, 0x71, 0x3c, 0x74, 0xe4, 0x34, 0x41, 0xed,
0x22, 0xc3, 0x93, 0xbe, 0x9b, 0xb5, 0x30, 0xbf, 0x6a, 0xe1, 0x5d, 0xa8, 0xc4, 0x2c, 0xfa, 0x74, 0x46, 0xb3, 0x7b, 0xf3, 0x1c, 0x88, 0x36, 0x1e, 0x1e, 0x4c, 0x13, 0xb4, 0xab, 0x3c, 0xfd, 0x20,
0x2a, 0x8f, 0x15, 0xd4, 0xb1, 0xb2, 0x92, 0xfb, 0x2e, 0x79, 0x07, 0x4a, 0xdc, 0x19, 0x63, 0x60, 0x3b, 0x50, 0x51, 0x47, 0xe7, 0x54, 0x97, 0x39, 0x1e, 0xf6, 0xfd, 0xbc, 0x87, 0xc5, 0x55, 0x0f,
0xb7, 0x8a, 0x8a, 0x8f, 0xbb, 0x97, 0xf2, 0xf1, 0xd4, 0x8f, 0x86, 0x66, 0xba, 0xd1, 0x78, 0xae, 0xaf, 0xc3, 0x5a, 0xc2, 0xd9, 0xc7, 0x53, 0x75, 0xac, 0xa4, 0x8f, 0x55, 0xb5, 0xdc, 0xf7, 0xc9,
0x41, 0xb3, 0xc7, 0xa2, 0xf8, 0x46, 0xe3, 0x3a, 0x80, 0x75, 0x67, 0x6e, 0x9f, 0x15, 0xda, 0x01, 0x5b, 0x50, 0x11, 0xde, 0x18, 0x23, 0xb7, 0x55, 0xd6, 0x7c, 0x5c, 0x3f, 0x93, 0x8f, 0xfb, 0x21,
0xa6, 0x00, 0xdf, 0xcc, 0x5a, 0x94, 0x86, 0x68, 0x67, 0x01, 0xe6, 0xd0, 0x0e, 0xd0, 0x6c, 0x38, 0x1b, 0xda, 0xd9, 0x46, 0xeb, 0x99, 0x01, 0x3b, 0x3d, 0xce, 0x92, 0x4b, 0x8d, 0xeb, 0x31, 0x6c,
0x19, 0xd9, 0xf8, 0x5d, 0x83, 0xad, 0x7d, 0x9b, 0xdf, 0x26, 0xc8, 0x7f, 0x6a, 0x70, 0xb7, 0x87, 0x78, 0x73, 0xff, 0x9c, 0xd8, 0x8d, 0x30, 0x03, 0xf8, 0x7a, 0xde, 0xa3, 0xac, 0xf8, 0x3a, 0x0b,
0xdc, 0x61, 0x74, 0x88, 0xb7, 0x09, 0xf7, 0xd7, 0x1a, 0x34, 0x07, 0xe3, 0xe8, 0xec, 0x26, 0x63, 0x30, 0x4f, 0xdc, 0x08, 0xed, 0xa6, 0x97, 0x93, 0xad, 0x5f, 0x0d, 0xd8, 0x7e, 0xe8, 0x8a, 0xab,
0x36, 0x7e, 0xd3, 0xe0, 0x95, 0xa4, 0xba, 0x1c, 0xd9, 0x4c, 0xd0, 0x1b, 0xea, 0x99, 0xf7, 0xa1, 0x04, 0xf9, 0x77, 0x03, 0xae, 0xf7, 0x50, 0x78, 0x9c, 0x0e, 0xf1, 0x2a, 0xe1, 0xfe, 0xd2, 0x80,
0x11, 0xcf, 0xcc, 0x5b, 0x76, 0xcc, 0xc3, 0xcb, 0x1d, 0x33, 0x87, 0xa2, 0xfc, 0x52, 0x8f, 0x97, 0x9d, 0xc1, 0x98, 0x1d, 0x5f, 0x66, 0xcc, 0xd6, 0x2f, 0x06, 0xbc, 0x94, 0x76, 0x97, 0xa7, 0x2e,
0x45, 0xe3, 0x57, 0x0d, 0xb6, 0x64, 0xd5, 0xb9, 0x2d, 0x78, 0x7f, 0xd1, 0x60, 0x73, 0xdf, 0xe6, 0x97, 0xf4, 0x92, 0x46, 0xe6, 0x5d, 0x68, 0x26, 0x33, 0xf7, 0x96, 0x03, 0x73, 0xfb, 0xec, 0xc0,
0xb7, 0x05, 0xee, 0x73, 0x0d, 0x5a, 0xb3, 0x6a, 0x73, 0x5b, 0x30, 0xcb, 0x1f, 0x15, 0x59, 0x69, 0xcc, 0xa1, 0xe8, 0xb8, 0x34, 0x92, 0x65, 0xd1, 0xfa, 0xd9, 0x80, 0x6d, 0xd5, 0x75, 0xae, 0x0a,
0x6e, 0x32, 0xde, 0xff, 0xb9, 0xb8, 0xbe, 0xc8, 0x41, 0xbd, 0x1f, 0x72, 0x64, 0xe2, 0xe5, 0x61, 0xde, 0x9f, 0x0c, 0xd8, 0x7a, 0xe8, 0x8a, 0xab, 0x02, 0xf7, 0x99, 0x01, 0xad, 0x59, 0xb7, 0xb9,
0x7d, 0xeb, 0xa2, 0xc9, 0x12, 0x71, 0x75, 0xd5, 0x18, 0xf2, 0x10, 0x16, 0x0e, 0xb1, 0x84, 0x3d, 0x2a, 0x98, 0xd5, 0x8f, 0x8a, 0xea, 0x34, 0x97, 0x19, 0xef, 0x7f, 0xdc, 0x5c, 0x9f, 0x17, 0xa0,
0x52, 0xd8, 0xab, 0xe6, 0xda, 0x5c, 0x79, 0x6c, 0x8f, 0xc8, 0xeb, 0x00, 0x1c, 0x47, 0x01, 0x86, 0xd1, 0x8f, 0x05, 0x72, 0xf9, 0xe2, 0xb0, 0xbe, 0x71, 0xda, 0x65, 0x85, 0xb8, 0xb6, 0xea, 0x0c,
0x42, 0x3e, 0x54, 0x54, 0x0f, 0x55, 0x53, 0x4d, 0xdf, 0x95, 0xcb, 0xce, 0xd8, 0x0e, 0x43, 0xf4, 0xb9, 0x0d, 0x8b, 0x80, 0x38, 0xd2, 0x1d, 0x69, 0xec, 0x35, 0x7b, 0x7d, 0xae, 0x3c, 0x70, 0x47,
0xe5, 0x72, 0x29, 0x59, 0x4e, 0x35, 0x7d, 0x37, 0xc3, 0x6c, 0x39, 0xcb, 0xec, 0x7d, 0x80, 0xb9, 0xe4, 0x55, 0x00, 0x81, 0xa3, 0x08, 0x63, 0xa9, 0x0c, 0x95, 0xb5, 0xa1, 0x5a, 0xa6, 0xe9, 0xfb,
0x07, 0x78, 0xab, 0xd2, 0xce, 0xef, 0x14, 0xcc, 0x25, 0x8d, 0x6c, 0x8e, 0x59, 0x74, 0x66, 0x51, 0x6a, 0xd9, 0x1b, 0xbb, 0x71, 0x8c, 0xa1, 0x5a, 0xae, 0xa4, 0xcb, 0x99, 0xa6, 0xef, 0xe7, 0x98,
0x97, 0xb7, 0xaa, 0xed, 0xbc, 0x6c, 0x8e, 0x59, 0x74, 0xd6, 0x77, 0x39, 0x79, 0x17, 0x2a, 0x72, 0xad, 0xe6, 0x99, 0xbd, 0x09, 0x30, 0x8f, 0x80, 0x68, 0xad, 0xb5, 0x8b, 0x7b, 0x25, 0x7b, 0x49,
0xc1, 0xb5, 0x85, 0xdd, 0x82, 0x76, 0xfe, 0xfa, 0xa6, 0x4d, 0xde, 0xd1, 0xb3, 0x85, 0x6d, 0x7c, 0xa3, 0x1e, 0xc7, 0x9c, 0x1d, 0x3b, 0xd4, 0x17, 0xad, 0x5a, 0xbb, 0xa8, 0x1e, 0xc7, 0x9c, 0x1d,
0x96, 0x83, 0x7a, 0x0f, 0x7d, 0x14, 0x78, 0x03, 0x98, 0xcf, 0xb2, 0x56, 0xb8, 0x8e, 0xb5, 0xe2, 0xf7, 0x7d, 0x41, 0xde, 0x86, 0x35, 0xb5, 0xe0, 0xbb, 0xd2, 0x6d, 0x41, 0xbb, 0x78, 0xf1, 0xa3,
0x75, 0xac, 0x95, 0x2e, 0xb0, 0xf6, 0x06, 0xac, 0xc5, 0x8c, 0x06, 0x36, 0x9b, 0x5a, 0x1e, 0x4e, 0x4d, 0xdd, 0xd1, 0x73, 0xa5, 0x6b, 0x7d, 0x52, 0x80, 0x46, 0x0f, 0x43, 0x94, 0x78, 0x09, 0x98,
0x79, 0xab, 0xac, 0xa8, 0xab, 0xa5, 0xba, 0x67, 0x38, 0xe5, 0xc6, 0x0b, 0x0d, 0xea, 0x03, 0xb4, 0xcf, 0xb3, 0x56, 0xba, 0x88, 0xb5, 0xf2, 0x45, 0xac, 0x55, 0x4e, 0xb1, 0xf6, 0x1a, 0xac, 0x27,
0x99, 0x33, 0x7e, 0x79, 0x4c, 0x2c, 0x23, 0xc8, 0x67, 0x11, 0x64, 0x52, 0xb1, 0xb0, 0x9a, 0x8a, 0x9c, 0x46, 0x2e, 0x9f, 0x3a, 0x01, 0x4e, 0x45, 0xab, 0xaa, 0xa9, 0xab, 0x67, 0xba, 0x47, 0x38,
0x8f, 0x60, 0x83, 0x21, 0x9f, 0xf8, 0xc2, 0x5a, 0x22, 0x28, 0xe1, 0x60, 0x3d, 0x59, 0xd8, 0x9b, 0x15, 0xd6, 0x73, 0x03, 0x1a, 0x03, 0x74, 0xb9, 0x37, 0x7e, 0x71, 0x4c, 0x2c, 0x23, 0x28, 0xe6,
0xd3, 0xb4, 0x0b, 0xc5, 0x93, 0x09, 0xb2, 0xa9, 0x0a, 0xbb, 0x6b, 0xa3, 0x20, 0xd9, 0x67, 0x7c, 0x11, 0xe4, 0x4a, 0xb1, 0xb4, 0x5a, 0x8a, 0x77, 0x60, 0x93, 0xa3, 0x98, 0x84, 0xd2, 0x59, 0x22,
0x99, 0x83, 0xb5, 0x19, 0x72, 0x79, 0xd5, 0x7f, 0x9b, 0x86, 0xfe, 0x3d, 0x64, 0x03, 0xea, 0xca, 0x28, 0xe5, 0x60, 0x23, 0x5d, 0x78, 0x30, 0xa7, 0x69, 0x1f, 0xca, 0x87, 0x13, 0xe4, 0x53, 0x9d,
0x00, 0x2b, 0x8c, 0x5c, 0x5c, 0x78, 0xbc, 0xa6, 0x94, 0x87, 0x91, 0x8b, 0xab, 0xb4, 0x14, 0xff, 0x76, 0x17, 0x66, 0x41, 0xba, 0xcf, 0xfa, 0xbc, 0x00, 0xeb, 0x33, 0xe4, 0xea, 0xaa, 0x7f, 0x36,
0x11, 0x2d, 0xa5, 0xcb, 0x69, 0xe9, 0x40, 0x61, 0x4c, 0x45, 0xe2, 0xfa, 0x5a, 0xf7, 0xde, 0xe5, 0x0d, 0xfd, 0x7d, 0xc8, 0x16, 0x34, 0xb4, 0x03, 0x4e, 0xcc, 0x7c, 0x5c, 0x44, 0xbc, 0xae, 0x95,
0x75, 0x6a, 0x9f, 0x0a, 0x6e, 0xaa, 0x7d, 0x46, 0x0f, 0x6a, 0xc7, 0x34, 0xc0, 0xc1, 0x34, 0x74, 0x4f, 0x98, 0x8f, 0xab, 0xb4, 0x94, 0xff, 0x12, 0x2d, 0x95, 0xb3, 0x69, 0xe9, 0x40, 0x69, 0x4c,
0x0e, 0xf8, 0xe8, 0xea, 0xa1, 0xf4, 0xda, 0x29, 0xd0, 0xf8, 0x4a, 0x83, 0xf2, 0x33, 0x9c, 0x76, 0x65, 0x1a, 0xfa, 0x7a, 0xf7, 0xc6, 0xd9, 0x7d, 0xea, 0x21, 0x95, 0xc2, 0xd6, 0xfb, 0xac, 0x1e,
0x07, 0x38, 0x52, 0x0c, 0xa9, 0xd4, 0x4d, 0x6f, 0x28, 0xaa, 0xcc, 0x25, 0x0f, 0xa0, 0xb6, 0x14, 0xd4, 0x0f, 0x68, 0x84, 0x83, 0x69, 0xec, 0x3d, 0x16, 0xa3, 0xf3, 0x87, 0xd2, 0x0b, 0xa7, 0x40,
0x9b, 0x29, 0x7b, 0xb0, 0x08, 0xcd, 0xbf, 0xaf, 0xd2, 0x94, 0x5b, 0xa7, 0xb6, 0x9f, 0x12, 0x58, 0xeb, 0x0b, 0x03, 0xaa, 0x8f, 0x70, 0xda, 0x1d, 0xe0, 0x48, 0x33, 0xa4, 0x4b, 0x37, 0xbb, 0xa1,
0x31, 0xcb, 0x94, 0x7f, 0x20, 0x45, 0x79, 0xf3, 0xa2, 0x48, 0xf1, 0x56, 0x51, 0x05, 0x3d, 0xcc, 0xac, 0x2b, 0x97, 0xdc, 0x82, 0xfa, 0x52, 0x6e, 0x66, 0xec, 0xc1, 0x22, 0x35, 0xff, 0xbc, 0x4b,
0xab, 0x14, 0x37, 0x3e, 0x06, 0x48, 0x8d, 0x93, 0x10, 0x17, 0x1e, 0xd4, 0x96, 0x3d, 0xf8, 0x1e, 0x53, 0xe1, 0x1c, 0xb9, 0x61, 0x46, 0xe0, 0x9a, 0x5d, 0xa5, 0xe2, 0x3d, 0x25, 0xaa, 0x9b, 0x17,
0x94, 0x3d, 0x9c, 0x76, 0x39, 0x8e, 0x5a, 0x39, 0xc5, 0xdd, 0x55, 0x59, 0x90, 0x5e, 0x65, 0xce, 0x4d, 0x4a, 0xb4, 0xca, 0x3a, 0xe9, 0x61, 0xde, 0xa5, 0x84, 0xf5, 0x21, 0x40, 0xe6, 0x9c, 0x82,
0xb6, 0x1b, 0x21, 0x6c, 0x0c, 0x92, 0xc7, 0x64, 0xac, 0x50, 0x2e, 0xa8, 0xc3, 0x57, 0x2a, 0xa7, 0xb8, 0x88, 0xa0, 0xb1, 0x1c, 0xc1, 0x77, 0xa0, 0x1a, 0xe0, 0xb4, 0x2b, 0x70, 0xd4, 0x2a, 0x68,
0xb6, 0x5a, 0x39, 0x1f, 0x40, 0x2d, 0xc0, 0x20, 0x62, 0x53, 0x8b, 0xd3, 0x73, 0x9c, 0xb1, 0x91, 0xee, 0xce, 0xab, 0x82, 0xec, 0x2a, 0x7b, 0xb6, 0xdd, 0x8a, 0x61, 0x73, 0x90, 0x1a, 0x53, 0xb9,
0xa8, 0x06, 0xf4, 0x1c, 0x25, 0xde, 0x70, 0x12, 0x58, 0x2c, 0x3a, 0xe3, 0xb3, 0x80, 0x0a, 0x27, 0x42, 0x85, 0xa4, 0x9e, 0x58, 0xe9, 0x9c, 0xc6, 0x6a, 0xe7, 0xbc, 0x05, 0xf5, 0x08, 0x23, 0xc6,
0x81, 0x19, 0x9d, 0xf1, 0x47, 0xdf, 0xe4, 0xa0, 0x9c, 0xa6, 0x22, 0xa9, 0x42, 0xd1, 0x3b, 0x8c, 0xa7, 0x8e, 0xa0, 0x27, 0x38, 0x63, 0x23, 0x55, 0x0d, 0xe8, 0x09, 0x2a, 0xbc, 0xf1, 0x24, 0x72,
0x42, 0xd4, 0xef, 0x90, 0x26, 0x6c, 0x78, 0xab, 0x33, 0xb7, 0xee, 0x92, 0x4d, 0x58, 0xf7, 0xb2, 0x38, 0x3b, 0x16, 0xb3, 0x84, 0x8a, 0x27, 0x91, 0xcd, 0x8e, 0xc5, 0x9d, 0xaf, 0x0a, 0x50, 0xcd,
0x03, 0xab, 0x8e, 0x84, 0x40, 0xc3, 0xcb, 0x4c, 0x74, 0xfa, 0x27, 0x64, 0x1b, 0x36, 0xbd, 0x8b, 0x4a, 0x91, 0xd4, 0xa0, 0x1c, 0x3c, 0x61, 0x31, 0x9a, 0xd7, 0xc8, 0x0e, 0x6c, 0x06, 0xab, 0x33,
0x23, 0x8f, 0x3e, 0x22, 0x5b, 0xa0, 0x7b, 0xd9, 0x99, 0x80, 0xeb, 0x63, 0xd2, 0x04, 0xdd, 0x5b, 0xb7, 0xe9, 0x93, 0x2d, 0xd8, 0x08, 0xf2, 0x03, 0xab, 0x89, 0x84, 0x40, 0x33, 0xc8, 0x4d, 0x74,
0x69, 0xc2, 0xf5, 0xef, 0x34, 0xb2, 0x09, 0x0d, 0x2f, 0xd3, 0xa9, 0xea, 0xdf, 0x6b, 0x84, 0x40, 0xe6, 0x47, 0x64, 0x17, 0xb6, 0x82, 0xd3, 0x23, 0x8f, 0x39, 0x22, 0xdb, 0x60, 0x06, 0xf9, 0x99,
0xdd, 0x5b, 0x6e, 0xe7, 0xf4, 0x1f, 0x34, 0xb2, 0x0d, 0xc4, 0xbb, 0xd0, 0xf3, 0xe8, 0x3f, 0x6a, 0x40, 0x98, 0x63, 0xb2, 0x03, 0x66, 0xb0, 0xf2, 0x08, 0x37, 0xbf, 0x31, 0xc8, 0x16, 0x34, 0x83,
0x64, 0x0b, 0xd6, 0xbd, 0x4c, 0x63, 0xc0, 0xf5, 0x9f, 0x34, 0xb2, 0x06, 0x65, 0x2f, 0xf9, 0xed, 0xdc, 0x4b, 0xd5, 0xfc, 0xd6, 0x20, 0x04, 0x1a, 0xc1, 0xf2, 0x73, 0xce, 0xfc, 0xce, 0x20, 0xbb,
0xd4, 0x3f, 0xcf, 0x2b, 0x29, 0xa9, 0xe7, 0xfa, 0x17, 0x89, 0x94, 0x64, 0xb6, 0xfe, 0x47, 0x9e, 0x40, 0x82, 0x53, 0x6f, 0x1e, 0xf3, 0x7b, 0x83, 0x6c, 0xc3, 0x46, 0x90, 0x7b, 0x18, 0x08, 0xf3,
0x34, 0xa0, 0xea, 0xc9, 0x90, 0x3e, 0xa6, 0x8e, 0xa7, 0x7f, 0x5b, 0x7d, 0xf4, 0x04, 0x2a, 0xb3, 0x07, 0x83, 0xac, 0x43, 0x35, 0x48, 0x7f, 0x3b, 0xcd, 0x4f, 0x8b, 0x5a, 0x4a, 0xfb, 0xb9, 0xf9,
0xff, 0x4a, 0x08, 0x40, 0xe9, 0xc0, 0xe6, 0x02, 0x99, 0x7e, 0x47, 0x7e, 0x9b, 0x68, 0xbb, 0xc8, 0x59, 0x2a, 0xa5, 0x95, 0x6d, 0xfe, 0x56, 0x24, 0x4d, 0xa8, 0x05, 0x2a, 0xa5, 0x0f, 0xa8, 0x17,
0x74, 0x4d, 0x7e, 0x7f, 0xc8, 0xa8, 0xd4, 0xe7, 0x24, 0xa5, 0x47, 0x32, 0x75, 0xf5, 0xfc, 0xd3, 0x98, 0x5f, 0xd7, 0xee, 0xdc, 0x83, 0xb5, 0xd9, 0x7f, 0x25, 0x04, 0xa0, 0xf2, 0xd8, 0x15, 0x12,
0xde, 0x47, 0x4f, 0x47, 0x54, 0x8c, 0x27, 0x43, 0x59, 0x0c, 0x76, 0xcf, 0xa9, 0xef, 0xd3, 0x73, 0xb9, 0x79, 0x4d, 0x7d, 0xdb, 0xe8, 0xfa, 0xc8, 0x4d, 0x43, 0x7d, 0xbf, 0xcf, 0xa9, 0xd2, 0x17,
0x81, 0xce, 0x78, 0x37, 0x89, 0x8c, 0xb7, 0x5d, 0xca, 0x05, 0xa3, 0xc3, 0x89, 0x40, 0x77, 0x77, 0x14, 0xa5, 0x4f, 0x55, 0xe9, 0x9a, 0xc5, 0xfb, 0xbd, 0x0f, 0xee, 0x8f, 0xa8, 0x1c, 0x4f, 0x86,
0x16, 0x1f, 0xbb, 0x2a, 0x5c, 0xe6, 0x62, 0x3c, 0x1c, 0x96, 0x94, 0xe6, 0xf1, 0x5f, 0x01, 0x00, 0xaa, 0x19, 0xec, 0x9f, 0xd0, 0x30, 0xa4, 0x27, 0x12, 0xbd, 0xf1, 0x7e, 0x9a, 0x19, 0x6f, 0xfa,
0x00, 0xff, 0xff, 0x7a, 0xb3, 0x5a, 0xe2, 0x4b, 0x12, 0x00, 0x00, 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,
} }

View File

@ -89,5 +89,6 @@ service Master {
rpc ShowPartitions(internal.ShowPartitionRequest) returns (service.StringListResponse) {} 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) {}
} }

View File

@ -30,33 +30,34 @@ const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
func init() { proto.RegisterFile("master.proto", fileDescriptor_f9c348dec43a6705) } func init() { proto.RegisterFile("master.proto", fileDescriptor_f9c348dec43a6705) }
var fileDescriptor_f9c348dec43a6705 = []byte{ var fileDescriptor_f9c348dec43a6705 = []byte{
// 409 bytes of a gzipped FileDescriptorProto // 432 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x94, 0xcd, 0x6e, 0xe2, 0x30, 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x94, 0x51, 0xaf, 0xd2, 0x30,
0x14, 0x85, 0x41, 0x23, 0xa1, 0x91, 0xc5, 0xcf, 0xe0, 0xd9, 0x31, 0xab, 0xc9, 0x0a, 0xc1, 0x90, 0x14, 0xc7, 0x79, 0xba, 0xc6, 0x86, 0xcb, 0xf5, 0xd6, 0x37, 0x7c, 0xf1, 0xee, 0xc9, 0x80, 0x6c,
0xa0, 0xe9, 0x0b, 0x54, 0xc0, 0x82, 0x45, 0x2b, 0x55, 0xc0, 0xaa, 0x55, 0x45, 0x9d, 0x60, 0x81, 0x46, 0xbf, 0x80, 0x02, 0x0f, 0x90, 0x68, 0x42, 0x80, 0x17, 0x35, 0x06, 0xbb, 0xad, 0x81, 0xc6,
0xd5, 0x24, 0x4e, 0x7d, 0x6f, 0xa8, 0xc4, 0xfb, 0xf6, 0x3d, 0xaa, 0x24, 0x84, 0xc4, 0x05, 0x53, 0x6e, 0x9d, 0x3d, 0x67, 0x98, 0xf0, 0xe1, 0xfc, 0x6c, 0x66, 0x1b, 0xdd, 0x56, 0xa1, 0x88, 0xf7,
0xda, 0x1d, 0xb6, 0x8f, 0xbf, 0x83, 0xef, 0x39, 0x0a, 0xa9, 0x07, 0x0c, 0x90, 0x2b, 0x3b, 0x52, 0x8d, 0xb6, 0xff, 0xf3, 0xfb, 0x73, 0xce, 0xf9, 0x67, 0xa4, 0x9b, 0x30, 0x40, 0xae, 0xfd, 0x4c,
0x12, 0x25, 0xfd, 0x1d, 0x08, 0x7f, 0x1b, 0x43, 0xb6, 0xb2, 0xb3, 0xa3, 0x4e, 0xdd, 0x93, 0x41, 0x2b, 0x54, 0xf4, 0x79, 0x22, 0xe4, 0x3e, 0x87, 0xea, 0xe4, 0x57, 0x4f, 0xfd, 0x6e, 0xa4, 0x92,
0x20, 0xc3, 0x6c, 0xb3, 0x43, 0x45, 0x88, 0x5c, 0x85, 0xcc, 0x5f, 0x06, 0xb0, 0xde, 0xef, 0xb5, 0x44, 0xa5, 0xd5, 0x65, 0x9f, 0x8a, 0x14, 0xb9, 0x4e, 0x99, 0xdc, 0x24, 0xb0, 0x3d, 0xde, 0xdd,
0x81, 0xab, 0xad, 0xf0, 0x78, 0xb1, 0xf5, 0xff, 0xed, 0x27, 0xa9, 0xdd, 0xa6, 0xf7, 0x29, 0x23, 0x03, 0xd7, 0x7b, 0x11, 0xf1, 0xe6, 0xea, 0xed, 0xef, 0xa7, 0xe4, 0xe6, 0x53, 0x59, 0x4f, 0x19,
0xbf, 0xc6, 0x8a, 0x33, 0xe4, 0x63, 0xe9, 0xfb, 0xdc, 0x43, 0x21, 0x43, 0x6a, 0xdb, 0x9a, 0x53, 0x79, 0x36, 0xd1, 0x9c, 0x21, 0x9f, 0x28, 0x29, 0x79, 0x84, 0x42, 0xa5, 0xd4, 0xf7, 0x2d, 0x27,
0xce, 0xb4, 0x3f, 0x0a, 0x67, 0xfc, 0x25, 0xe6, 0x80, 0x9d, 0x3f, 0xba, 0x7e, 0xff, 0x8f, 0xe6, 0xc3, 0xf4, 0xff, 0x16, 0x2e, 0xf9, 0xcf, 0x9c, 0x03, 0xf6, 0x5f, 0xd8, 0xfa, 0xe3, 0x3f, 0x5a,
0xc8, 0x30, 0x06, 0xab, 0x42, 0x1f, 0x49, 0x73, 0xa2, 0x64, 0x54, 0x32, 0xf8, 0x67, 0x30, 0xd0, 0x21, 0xc3, 0x1c, 0xbc, 0x0e, 0xfd, 0x46, 0x7a, 0x53, 0xad, 0xb2, 0x96, 0xc1, 0x6b, 0x87, 0x81,
0x65, 0x17, 0xe2, 0x5d, 0xd2, 0x98, 0x32, 0x28, 0xd1, 0xfb, 0x06, 0xba, 0xa6, 0xca, 0xe1, 0x96, 0x2d, 0xbb, 0x12, 0x1f, 0x92, 0xdb, 0x19, 0x83, 0x16, 0x7d, 0xe8, 0xa0, 0x5b, 0x2a, 0x03, 0xf7,
0x2e, 0xde, 0xcf, 0xca, 0x1e, 0x49, 0xe9, 0xcf, 0x38, 0x44, 0x32, 0x04, 0x6e, 0x55, 0x68, 0x4c, 0x6c, 0xf1, 0x71, 0x56, 0xfe, 0x58, 0x29, 0xb9, 0xe4, 0x90, 0xa9, 0x14, 0xb8, 0xd7, 0xa1, 0x39,
0xe8, 0x84, 0x83, 0xa7, 0x84, 0x5b, 0x9e, 0xd3, 0xd0, 0xf4, 0x8c, 0x23, 0x69, 0xee, 0xd6, 0x3f, 0xa1, 0x53, 0x0e, 0x91, 0x16, 0x61, 0x7b, 0x4e, 0x6f, 0x5c, 0x6d, 0x9c, 0x48, 0x8d, 0xdb, 0xf0,
0xed, 0x56, 0x08, 0xb3, 0xab, 0x51, 0xf2, 0xd3, 0xaa, 0xd0, 0x67, 0xd2, 0x9a, 0x6f, 0xe4, 0x6b, 0xbc, 0x5b, 0x23, 0xac, 0x4a, 0xb3, 0xe2, 0xa7, 0xd7, 0xa1, 0x3f, 0xc8, 0xdd, 0x6a, 0xa7, 0x7e,
0x71, 0x0c, 0xc6, 0xd1, 0xe9, 0xba, 0xdc, 0xaf, 0x7b, 0xda, 0x6f, 0x8e, 0x4a, 0x84, 0xeb, 0x1b, 0x35, 0xcf, 0xe0, 0x1c, 0x9d, 0xad, 0x33, 0x7e, 0xaf, 0xce, 0xfb, 0xad, 0x50, 0x8b, 0x74, 0xfb,
0x01, 0x58, 0x7a, 0xe3, 0x92, 0xb4, 0xb2, 0x80, 0xef, 0x98, 0x42, 0x91, 0x3e, 0x70, 0x70, 0xb6, 0x51, 0x00, 0xb6, 0x7a, 0xdc, 0x90, 0xbb, 0x6a, 0xc1, 0x0b, 0xa6, 0x51, 0x94, 0x0d, 0x8e, 0x2e,
0x08, 0x07, 0xdd, 0x85, 0x41, 0x3d, 0x90, 0x46, 0x12, 0x70, 0x81, 0xef, 0x9f, 0xa9, 0xc1, 0x57, 0x06, 0xa1, 0xd6, 0x5d, 0xb9, 0xa8, 0xaf, 0xe4, 0xb6, 0x58, 0x70, 0x83, 0x1f, 0x5e, 0x88, 0xc1,
0xe1, 0x4f, 0xa4, 0x3e, 0x65, 0x50, 0xb0, 0x7b, 0xe6, 0x12, 0x1c, 0xa1, 0x2f, 0xeb, 0x80, 0x22, 0xff, 0xc2, 0xbf, 0x93, 0xee, 0x8c, 0x41, 0xc3, 0x1e, 0xb8, 0x43, 0x70, 0x82, 0xbe, 0x2e, 0x03,
0xed, 0x3c, 0xd8, 0xc2, 0xc6, 0xf9, 0xa4, 0x02, 0x47, 0x5e, 0xbd, 0xd3, 0x5e, 0x07, 0x9d, 0x5e, 0x9a, 0xdc, 0x9b, 0xc5, 0x36, 0x36, 0xc1, 0x3f, 0x22, 0x70, 0xe2, 0x35, 0x38, 0xef, 0x55, 0xeb,
0x00, 0x41, 0x9a, 0x49, 0xb0, 0x87, 0x53, 0x30, 0xce, 0x4c, 0x93, 0x7d, 0x27, 0xfe, 0x19, 0xf9, 0xec, 0x00, 0x08, 0xd2, 0x2b, 0x16, 0x5b, 0xbf, 0x82, 0x73, 0x66, 0x96, 0xec, 0x31, 0xeb, 0xff,
0xb1, 0x00, 0x49, 0xff, 0x1a, 0xf8, 0x0b, 0x90, 0x86, 0x71, 0xe9, 0x92, 0x9c, 0xd7, 0xad, 0x0e, 0x4c, 0x7a, 0x1f, 0xa4, 0x54, 0xd1, 0x5a, 0x24, 0x1c, 0x90, 0x25, 0x19, 0x7d, 0x70, 0x58, 0xad,
0xab, 0xa3, 0xd1, 0xfd, 0xf5, 0x5a, 0xe0, 0x26, 0x76, 0x93, 0xb8, 0x9c, 0x9d, 0xf0, 0x7d, 0xb1, 0x41, 0x39, 0x26, 0x67, 0x4b, 0x6a, 0xf4, 0x82, 0x3c, 0x29, 0xd1, 0xf3, 0x98, 0xbe, 0x74, 0x14,
0x43, 0xee, 0x6d, 0x9c, 0x0c, 0x30, 0x58, 0x09, 0x40, 0x25, 0xdc, 0x18, 0xf9, 0xca, 0xc9, 0x31, 0xcc, 0x63, 0x83, 0x7c, 0xb8, 0xa0, 0x30, 0xc4, 0xf1, 0xf8, 0xcb, 0xfb, 0xad, 0xc0, 0x5d, 0x1e,
0x4e, 0x4a, 0x75, 0xb2, 0xcf, 0x5b, 0xe4, 0xba, 0xb5, 0x74, 0x7d, 0xf5, 0x1e, 0x00, 0x00, 0xff, 0x16, 0x39, 0x08, 0x0e, 0x42, 0x4a, 0x71, 0x40, 0x1e, 0xed, 0x82, 0xaa, 0x76, 0x14, 0x0b, 0x40,
0xff, 0x47, 0x62, 0x7a, 0xee, 0x0c, 0x05, 0x00, 0x00, 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. // Reference imports to suppress errors if they are not otherwise used.
@ -131,7 +132,8 @@ type MasterClient interface {
// //
// @return StringListResponse // @return StringListResponse
ShowPartitions(ctx context.Context, in *internalpb.ShowPartitionRequest, opts ...grpc.CallOption) (*servicepb.StringListResponse, error) 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 { type masterClient struct {
@ -232,35 +234,22 @@ func (c *masterClient) ShowPartitions(ctx context.Context, in *internalpb.ShowPa
return out, nil return out, nil
} }
func (c *masterClient) Tso(ctx context.Context, opts ...grpc.CallOption) (Master_TsoClient, error) { func (c *masterClient) AllocTimestamp(ctx context.Context, in *internalpb.TsoRequest, opts ...grpc.CallOption) (*internalpb.TsoResponse, error) {
stream, err := c.cc.NewStream(ctx, &_Master_serviceDesc.Streams[0], "/milvus.proto.master.Master/Tso", opts...) out := new(internalpb.TsoResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.master.Master/AllocTimestamp", in, out, opts...)
if err != nil { if err != nil {
return nil, err return nil, err
} }
x := &masterTsoClient{stream} return out, nil
return x, nil
} }
type Master_TsoClient interface { func (c *masterClient) AllocId(ctx context.Context, in *internalpb.IdRequest, opts ...grpc.CallOption) (*internalpb.IdResponse, error) {
Send(*internalpb.TsoRequest) error out := new(internalpb.IdResponse)
Recv() (*internalpb.TsoResponse, error) err := c.cc.Invoke(ctx, "/milvus.proto.master.Master/AllocId", in, out, opts...)
grpc.ClientStream if err != nil {
}
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 {
return nil, err return nil, err
} }
return m, nil return out, nil
} }
// MasterServer is the server API for Master service. // MasterServer is the server API for Master service.
@ -325,7 +314,8 @@ type MasterServer interface {
// //
// @return StringListResponse // @return StringListResponse
ShowPartitions(context.Context, *internalpb.ShowPartitionRequest) (*servicepb.StringListResponse, error) 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. // 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) { func (*UnimplementedMasterServer) ShowPartitions(ctx context.Context, req *internalpb.ShowPartitionRequest) (*servicepb.StringListResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method ShowPartitions not implemented") return nil, status.Errorf(codes.Unimplemented, "method ShowPartitions not implemented")
} }
func (*UnimplementedMasterServer) Tso(srv Master_TsoServer) error { func (*UnimplementedMasterServer) AllocTimestamp(ctx context.Context, req *internalpb.TsoRequest) (*internalpb.TsoResponse, error) {
return status.Errorf(codes.Unimplemented, "method Tso not implemented") 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) { 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) return interceptor(ctx, in, info, handler)
} }
func _Master_Tso_Handler(srv interface{}, stream grpc.ServerStream) error { func _Master_AllocTimestamp_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
return srv.(MasterServer).Tso(&masterTsoServer{stream}) in := new(internalpb.TsoRequest)
} if err := dec(in); err != nil {
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 {
return nil, err 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{ var _Master_serviceDesc = grpc.ServiceDesc{
@ -620,14 +623,15 @@ var _Master_serviceDesc = grpc.ServiceDesc{
MethodName: "ShowPartitions", MethodName: "ShowPartitions",
Handler: _Master_ShowPartitions_Handler, Handler: _Master_ShowPartitions_Handler,
}, },
},
Streams: []grpc.StreamDesc{
{ {
StreamName: "Tso", MethodName: "AllocTimestamp",
Handler: _Master_Tso_Handler, Handler: _Master_AllocTimestamp_Handler,
ServerStreams: true, },
ClientStreams: true, {
MethodName: "AllocId",
Handler: _Master_AllocId_Handler,
}, },
}, },
Streams: []grpc.StreamDesc{},
Metadata: "master.proto", Metadata: "master.proto",
} }

View File

@ -2,14 +2,15 @@ package proxy
import ( import (
"fmt" "fmt"
"log"
"sync"
"github.com/apache/pulsar-client-go/pulsar" "github.com/apache/pulsar-client-go/pulsar"
"github.com/golang/protobuf/proto" "github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/errors" "github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb" "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
pb "github.com/zilliztech/milvus-distributed/internal/proto/message" pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil" "github.com/zilliztech/milvus-distributed/internal/util/typeutil"
"log"
"sync"
) )
type manipulationReq struct { 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 func (req *manipulationReq) WaitToFinish() commonpb.Status { // wait until send into pulsar
req.wg.Wait() req.wg.Wait()
for _, stat := range req.stats{ for _, stat := range req.stats {
if stat.ErrorCode != commonpb.ErrorCode_SUCCESS{ if stat.ErrorCode != commonpb.ErrorCode_SUCCESS {
return stat return stat
} }
} }

View File

@ -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
}

View File

@ -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
}

View File

@ -3,6 +3,8 @@ package proxy
import ( import (
"context" "context"
"fmt" "fmt"
"strconv"
"github.com/apache/pulsar-client-go/pulsar" "github.com/apache/pulsar-client-go/pulsar"
"github.com/zilliztech/milvus-distributed/internal/allocator" "github.com/zilliztech/milvus-distributed/internal/allocator"
"github.com/zilliztech/milvus-distributed/internal/conf" "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/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil" "github.com/zilliztech/milvus-distributed/internal/util/typeutil"
etcd "go.etcd.io/etcd/clientv3" etcd "go.etcd.io/etcd/clientv3"
"strconv"
) )
type BaseRequest interface { type BaseRequest interface {
@ -100,7 +101,7 @@ func StartProxy(opt *ProxyOptions) error {
// saveInterval: opt.tsoSaveInterval, // saveInterval: opt.tsoSaveInterval,
//} //}
//tso.Restart(opt.proxyId) //tso.Restart(opt.proxyId)
tso := allocator.NewTimestampAllocator() tso, _ := allocator.NewTimestampAllocator(opt.ctx)
/////////////////// proxy server /////////////////////////////// /////////////////// proxy server ///////////////////////////////
//readerTopics, send insert and delete message into these topics //readerTopics, send insert and delete message into these topics

View File

@ -4,27 +4,35 @@ import (
"context" "context"
"encoding/binary" "encoding/binary"
"encoding/json" "encoding/json"
"github.com/apache/pulsar-client-go/pulsar" "os"
"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"
"sort" "sort"
"strconv" "strconv"
"testing" "testing"
"time" "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 ( const (
tsoKeyPath string = "/timestampOracle" 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) { func TestProxyNode(t *testing.T) {
startTestMaster("localhost:11000", t) startTestMaster("localhost:11000", t)
@ -75,10 +83,10 @@ func TestProxyNode(t *testing.T) {
assert.Nil(t, err) assert.Nil(t, err)
curValue, err := testOpt.tso.AllocOne() curValue, err := testOpt.tso.AllocOne()
curTS, err := timeAllocator.AllocOne() curTS, err := timeAllocator.AllocOne()
assert.Equalf(t, err, nil, "%s", "allocator failed") 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) t.Logf("current time stamp = %d, saved time stamp = %d", curTime, value)
assert.GreaterOrEqual(t, curValue, value) assert.GreaterOrEqual(t, curValue, value)
assert.GreaterOrEqual(t, value, startTime) assert.GreaterOrEqual(t, value, startTime)
@ -163,7 +171,7 @@ func TestProxyNode(t *testing.T) {
if err := proto.Unmarshal(cm.Payload(), &tsm); err != nil { if err := proto.Unmarshal(cm.Payload(), &tsm); err != nil {
t.Fatal(err) t.Fatal(err)
} }
curT, _:= tsoutil.ParseTS(tsm.Timestamp) curT, _ := tsoutil.ParseTS(tsm.Timestamp)
t.Logf("time tick = %d", curT) t.Logf("time tick = %d", curT)
assert.Greater(t, curT, lastT) assert.Greater(t, curT, lastT)
lastT = curT lastT = curT
@ -252,7 +260,7 @@ func TestProxyNode(t *testing.T) {
physicalTime, _ := tsoutil.ParseTS(qm.Timestamp) physicalTime, _ := tsoutil.ParseTS(qm.Timestamp)
t.Logf("query time stamp = %d", physicalTime) t.Logf("query time stamp = %d", physicalTime)
assert.Greater(t,physicalTime, startTime) assert.Greater(t, physicalTime, startTime)
r1 := pb.QueryResult{ r1 := pb.QueryResult{
Status: &pb.Status{ErrorCode: pb.ErrorCode_SUCCESS}, Status: &pb.Status{ErrorCode: pb.ErrorCode_SUCCESS},

View File

@ -1,13 +1,14 @@
package proxy package proxy
import ( import (
"log"
"sync"
"github.com/apache/pulsar-client-go/pulsar" "github.com/apache/pulsar-client-go/pulsar"
"github.com/golang/protobuf/proto" "github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb" "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb" "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb" "github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
"log"
"sync"
) )
type queryReq struct { type queryReq struct {
@ -80,7 +81,6 @@ func (s *proxyServer) restartQueryRoutine(buf_size int) error {
} }
qm.Timestamp = uint64(ts[0]) qm.Timestamp = uint64(ts[0])
qb, err := proto.Marshal(qm) qb, err := proto.Marshal(qm)
if err != nil { if err != nil {
log.Printf("Marshal QueryReqMsg failed, error = %v", err) log.Printf("Marshal QueryReqMsg failed, error = %v", err)
@ -141,22 +141,22 @@ func (s *proxyServer) reduceResults(query *queryReq) *servicepb.QueryResult {
status = *r.Status status = *r.Status
if status.ErrorCode == commonpb.ErrorCode_SUCCESS { if status.ErrorCode == commonpb.ErrorCode_SUCCESS {
results = append(results, r) results = append(results, r)
}else{ } else {
break break
} }
} }
if len(results) != s.numReaderNode{ if len(results) != s.numReaderNode {
status.ErrorCode = commonpb.ErrorCode_UNEXPECTED_ERROR status.ErrorCode = commonpb.ErrorCode_UNEXPECTED_ERROR
} }
if status.ErrorCode != commonpb.ErrorCode_SUCCESS{ if status.ErrorCode != commonpb.ErrorCode_SUCCESS {
result:= servicepb.QueryResult{ result := servicepb.QueryResult{
Status: &status, Status: &status,
} }
return &result return &result
} }
if s.numReaderNode == 1 { if s.numReaderNode == 1 {
result:= servicepb.QueryResult{ result := servicepb.QueryResult{
Status: &commonpb.Status{ Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS, ErrorCode: commonpb.ErrorCode_SUCCESS,
}, },

View File

@ -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
}
}

View File

@ -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)
}
}
}

View File

@ -1,8 +1,9 @@
package proxy package proxy
import ( import (
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
"sync" "sync"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
) )
type requestScheduler struct { type requestScheduler struct {

View File

@ -4,6 +4,12 @@ import (
"context" "context"
"encoding/binary" "encoding/binary"
"encoding/json" "encoding/json"
"net"
"sort"
"testing"
"time"
"unsafe"
"github.com/apache/pulsar-client-go/pulsar" "github.com/apache/pulsar-client-go/pulsar"
"github.com/golang/protobuf/proto" "github.com/golang/protobuf/proto"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
@ -13,11 +19,6 @@ import (
"github.com/zilliztech/milvus-distributed/internal/util/typeutil" "github.com/zilliztech/milvus-distributed/internal/util/typeutil"
"go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/clientv3"
"google.golang.org/grpc" "google.golang.org/grpc"
"net"
"sort"
"testing"
"time"
"unsafe"
) )
type testMasterServer struct { type testMasterServer struct {
@ -129,11 +130,11 @@ func TestProxyServer_WatchEtcd(t *testing.T) {
} }
seg2 := etcdpb.SegmentMeta{ seg2 := etcdpb.SegmentMeta{
SegmentId: 2, SegmentId: 2,
NumRows: 10, NumRows: 10,
} }
seg3 := etcdpb.SegmentMeta{ seg3 := etcdpb.SegmentMeta{
SegmentId: 3, SegmentId: 3,
NumRows: 10, NumRows: 10,
} }
if cb1, err := json.Marshal(&col1); err != nil { if cb1, err := json.Marshal(&col1); err != nil {
t.Fatal(err) t.Fatal(err)
@ -178,7 +179,7 @@ func TestProxyServer_WatchEtcd(t *testing.T) {
} }
seg5 := etcdpb.SegmentMeta{ seg5 := etcdpb.SegmentMeta{
SegmentId: 5, SegmentId: 5,
NumRows: 10, NumRows: 10,
} }
if cb4, err := json.Marshal(&col4); err != nil { if cb4, err := json.Marshal(&col4); err != nil {
t.Fatal(err) t.Fatal(err)

43
internal/proxy/task.go Normal file
View File

@ -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
}

View File

@ -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
}

View File

@ -2,13 +2,14 @@ package proxy
import ( import (
"context" "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" "log"
"time" "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 { type timeTick struct {
@ -49,10 +50,10 @@ func (tt *timeTick) tick() error {
return nil return nil
} }
func (tt *timeTick) Restart() error{ func (tt *timeTick) Restart() error {
tt.lastTick = 0 tt.lastTick = 0
ts, err := tt.getTimestamp() ts, err := tt.getTimestamp()
if err != nil{ if err != nil {
return err return err
} }

View File

@ -2,21 +2,22 @@ package proxy
import ( import (
"context" "context"
"testing"
"time"
"github.com/apache/pulsar-client-go/pulsar" "github.com/apache/pulsar-client-go/pulsar"
"github.com/golang/protobuf/proto" "github.com/golang/protobuf/proto"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
pb "github.com/zilliztech/milvus-distributed/internal/proto/message" pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil" "github.com/zilliztech/milvus-distributed/internal/util/typeutil"
"testing"
"time"
) )
func TestTimeTick(t *testing.T) { func TestTimeTick(t *testing.T) {
client, err := pulsar.NewClient(pulsar.ClientOptions{URL: "pulsar://localhost:6650"}) 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"}) producer, err := client.CreateProducer(pulsar.ProducerOptions{Topic: "timesync"})
assert.Nil(t,err) assert.Nil(t, err)
consumer, err := client.Subscribe(pulsar.ConsumerOptions{ consumer, err := client.Subscribe(pulsar.ConsumerOptions{
Topic: "timesync", Topic: "timesync",
@ -24,7 +25,7 @@ func TestTimeTick(t *testing.T) {
Type: pulsar.KeyShared, Type: pulsar.KeyShared,
SubscriptionInitialPosition: pulsar.SubscriptionPositionEarliest, SubscriptionInitialPosition: pulsar.SubscriptionPositionEarliest,
}) })
assert.Nil(t,err) assert.Nil(t, err)
ctx, _ := context.WithTimeout(context.Background(), 4*time.Second) ctx, _ := context.WithTimeout(context.Background(), 4*time.Second)

View File

@ -2,8 +2,9 @@ package reader
import ( import (
"context" "context"
"github.com/stretchr/testify/assert"
"testing" "testing"
"github.com/stretchr/testify/assert"
) )
func TestCollection_NewPartition(t *testing.T) { func TestCollection_NewPartition(t *testing.T) {

View File

@ -3,10 +3,11 @@ package reader
import ( import (
"context" "context"
"fmt" "fmt"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/zilliztech/milvus-distributed/internal/util/flowgraph"
"log" "log"
"sync" "sync"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/zilliztech/milvus-distributed/internal/util/flowgraph"
) )
type manipulationService struct { type manipulationService struct {

View File

@ -3,15 +3,16 @@ package reader
import ( import (
"context" "context"
"encoding/binary" "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" "math"
"strconv" "strconv"
"sync" "sync"
"testing" "testing"
"time" "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) { func TestInsertAndDelete_MessagesPreprocess(t *testing.T) {

View File

@ -38,7 +38,7 @@ type insertMsg struct {
type deletePreprocessMsg struct { type deletePreprocessMsg struct {
deletePreprocessData DeletePreprocessData deletePreprocessData DeletePreprocessData
timeRange TimeRange timeRange TimeRange
} }
type deleteMsg struct { type deleteMsg struct {
@ -47,7 +47,7 @@ type deleteMsg struct {
} }
type serviceTimeMsg struct { type serviceTimeMsg struct {
timeRange TimeRange timeRange TimeRange
} }
type InsertData struct { type InsertData struct {

View File

@ -2,17 +2,18 @@ package reader
import ( import (
"context" "context"
"log"
"math"
"sync"
"testing"
"time"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/conf" "github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/master/collection" "github.com/zilliztech/milvus-distributed/internal/master/collection"
"github.com/zilliztech/milvus-distributed/internal/master/segment" "github.com/zilliztech/milvus-distributed/internal/master/segment"
"github.com/zilliztech/milvus-distributed/internal/msgclient" "github.com/zilliztech/milvus-distributed/internal/msgclient"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb" "github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"log"
"math"
"sync"
"testing"
"time"
) )
func TestMeta_GetCollectionObjId(t *testing.T) { func TestMeta_GetCollectionObjId(t *testing.T) {

View File

@ -2,8 +2,9 @@ package reader
import ( import (
"context" "context"
"github.com/stretchr/testify/assert"
"testing" "testing"
"github.com/stretchr/testify/assert"
) )
func TestPartition_NewSegment(t *testing.T) { func TestPartition_NewSegment(t *testing.T) {

View File

@ -1,8 +1,9 @@
package reader package reader
import ( import (
"github.com/stretchr/testify/assert"
"testing" "testing"
"github.com/stretchr/testify/assert"
) )
func TestQueryNodeTime_UpdateReadTimeSync(t *testing.T) { func TestQueryNodeTime_UpdateReadTimeSync(t *testing.T) {

View File

@ -2,9 +2,10 @@ package reader
import ( import (
"context" "context"
"testing"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/conf" "github.com/zilliztech/milvus-distributed/internal/conf"
"testing"
) )
func TestQueryNode_CreateQueryNode(t *testing.T) { func TestQueryNode_CreateQueryNode(t *testing.T) {

View File

@ -3,10 +3,11 @@ package reader
import ( import (
"context" "context"
"fmt" "fmt"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
"log" "log"
"sync" "sync"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
) )
func StartQueryNode(ctx context.Context, pulsarURL string) { func StartQueryNode(ctx context.Context, pulsarURL string) {

View File

@ -2,13 +2,14 @@ package reader
import ( import (
"context" "context"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
"strconv" "strconv"
"sync" "sync"
"testing" "testing"
"time" "time"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
) )
const ctxTimeInMillisecond = 10 const ctxTimeInMillisecond = 10

View File

@ -2,8 +2,9 @@ package reader
import ( import (
"context" "context"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb" "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" 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 { func (node *QueryNode) PublishFailedSearchResult() commonpb.Status {
var results = msgpb.QueryResult{ var results = msgpb.QueryResult{}
}
var ctx = context.Background() var ctx = context.Background()

View File

@ -2,13 +2,14 @@ package reader
import ( import (
"context" "context"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
"strconv" "strconv"
"testing" "testing"
"time" "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" //masterPb "github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb" "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message" msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"

View File

@ -2,9 +2,10 @@ package reader
import ( import (
"fmt" "fmt"
"sort"
"github.com/zilliztech/milvus-distributed/internal/conf" "github.com/zilliztech/milvus-distributed/internal/conf"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message" msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"sort"
) )
func (node *QueryNode) Search(searchMessages []*msgPb.SearchMsg) msgPb.Status { func (node *QueryNode) Search(searchMessages []*msgPb.SearchMsg) msgPb.Status {

View File

@ -3,14 +3,15 @@ package reader
import ( import (
"context" "context"
"encoding/binary" "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" "math"
"strconv" "strconv"
"sync" "sync"
"testing" "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 // NOTE: start pulsar before test

View File

@ -13,11 +13,12 @@ package reader
*/ */
import "C" import "C"
import ( import (
"strconv"
"unsafe"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/errors" "github.com/zilliztech/milvus-distributed/internal/errors"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message" msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"strconv"
"unsafe"
) )
const SegmentLifetime = 20000 const SegmentLifetime = 20000

View File

@ -2,11 +2,12 @@ package reader
import ( import (
"fmt" "fmt"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"log" "log"
"strconv" "strconv"
"time" "time"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
) )
//func (node *QueryNode) SegmentsManagement() { //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 // TODO: set master pb's segment id type from uint64 to int64
SegmentId: segmentID, SegmentId: segmentID,
MemorySize: currentMemSize, MemorySize: currentMemSize,
NumRows: segmentNumOfRows, NumRows: segmentNumOfRows,
} }
statisticData = append(statisticData, stat) statisticData = append(statisticData, stat)

View File

@ -2,11 +2,12 @@ package reader
import ( import (
"context" "context"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
"strconv" "strconv"
"testing" "testing"
"time" "time"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
) )
//func TestSegmentManagement_SegmentsManagement(t *testing.T) { //func TestSegmentManagement_SegmentsManagement(t *testing.T) {

View File

@ -4,10 +4,11 @@ import (
"encoding/json" "encoding/json"
"errors" "errors"
"fmt" "fmt"
log "github.com/apache/pulsar/pulsar-client-go/logutil"
"os" "os"
"strconv" "strconv"
"time" "time"
log "github.com/apache/pulsar/pulsar-client-go/logutil"
) )
// Function `GetSegmentByEntityId` should return entityIDs, timestamps and segmentIDs // Function `GetSegmentByEntityId` should return entityIDs, timestamps and segmentIDs

View File

@ -2,13 +2,14 @@ package reader
import ( import (
"context" "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" "strconv"
"testing" "testing"
"time" "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" "github.com/stretchr/testify/assert"
) )

View File

@ -2,15 +2,15 @@ package S3_driver_test
import ( import (
"context" "context"
s3_driver "github.com/zilliztech/milvus-distributed/internal/storage/internal/S3"
"github.com/stretchr/testify/assert"
"testing" "testing"
"github.com/stretchr/testify/assert"
s3_driver "github.com/zilliztech/milvus-distributed/internal/storage/internal/S3"
) )
var ctx = context.Background() var ctx = context.Background()
var client, err = s3_driver.NewS3Driver(ctx) var client, err = s3_driver.NewS3Driver(ctx)
func TestS3Driver_PutRowAndGetRow(t *testing.T) { func TestS3Driver_PutRowAndGetRow(t *testing.T) {
err = client.PutRow(ctx, []byte("bar"), []byte("abcdefghijklmnoopqrstuvwxyz"), "SegmentA", 1) err = client.PutRow(ctx, []byte("bar"), []byte("abcdefghijklmnoopqrstuvwxyz"), "SegmentA", 1)
assert.Nil(t, err) assert.Nil(t, err)
@ -30,7 +30,7 @@ func TestS3Driver_PutRowAndGetRow(t *testing.T) {
assert.Equal(t, "testkeybarorbar_1", string(object)) 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) err = client.DeleteRow(ctx, []byte("bar"), 5)
assert.Nil(t, err) assert.Nil(t, err)
object, _ := client.GetRow(ctx, []byte("bar"), 6) 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")} keys := [][]byte{[]byte("foo"), []byte("bar")}
values := [][]byte{[]byte("The key is foo!"), []byte("The key is bar!")} values := [][]byte{[]byte("The key is foo!"), []byte("The key is bar!")}
segments := []string{"segmentA", "segmentB"} segments := []string{"segmentA", "segmentB"}
@ -77,7 +77,7 @@ func TestS3Driver_PutRowsAndGetRows(t *testing.T){
assert.Equal(t, "The key is bar!", string(objects[1])) 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")} keys := [][]byte{[]byte("foo"), []byte("bar")}
timestamps := []uint64{3, 3} timestamps := []uint64{3, 3}
err := client.DeleteRows(ctx, keys, timestamps) err := client.DeleteRows(ctx, keys, timestamps)
@ -92,11 +92,11 @@ func TestS3Driver_DeleteRows(t *testing.T){
func TestS3Driver_PutLogAndGetLog(t *testing.T) { func TestS3Driver_PutLogAndGetLog(t *testing.T) {
err = client.PutLog(ctx, []byte("insert"), []byte("This is insert log!"), 1, 11) err = client.PutLog(ctx, []byte("insert"), []byte("This is insert log!"), 1, 11)
assert.Nil(t, err) 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) 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) 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) assert.Nil(t, err)
channels := []int{5, 8, 9, 10, 11, 12, 13} channels := []int{5, 8, 9, 10, 11, 12, 13}
@ -119,7 +119,7 @@ func TestS3Driver_Segment(t *testing.T) {
assert.Nil(t, err) 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!")) err := client.PutSegmentDL(ctx, "segmentB", []byte("This is segmentB's delete log!"))
assert.Nil(t, err) assert.Nil(t, err)
@ -129,4 +129,4 @@ func TestS3Driver_SegmentDL(t *testing.T){
err = client.DeleteSegmentDL(ctx, "segmentB") err = client.DeleteSegmentDL(ctx, "segmentB")
assert.Nil(t, err) assert.Nil(t, err)
} }

View File

@ -3,12 +3,13 @@ package S3_driver
import ( import (
"bytes" "bytes"
"context" "context"
"io"
"github.com/aws/aws-sdk-go/aws" "github.com/aws/aws-sdk-go/aws"
"github.com/aws/aws-sdk-go/aws/session" "github.com/aws/aws-sdk-go/aws/session"
"github.com/aws/aws-sdk-go/service/s3" "github.com/aws/aws-sdk-go/service/s3"
"github.com/zilliztech/milvus-distributed/internal/conf" "github.com/zilliztech/milvus-distributed/internal/conf"
. "github.com/zilliztech/milvus-distributed/internal/storage/type" . "github.com/zilliztech/milvus-distributed/internal/storage/type"
"io"
) )
var bucketName = conf.Config.Writer.Bucket 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) { func (s *S3Store) Get(ctx context.Context, key Key) (Value, error) {
object, err := s.client.GetObjectWithContext(ctx, &s3.GetObjectInput{ object, err := s.client.GetObjectWithContext(ctx, &s3.GetObjectInput{
Bucket: aws.String(bucketName), Bucket: aws.String(bucketName),
Key: aws.String(string(key)), Key: aws.String(string(key)),
}) })
if err != nil { if err != nil {
return nil, err return nil, err
@ -85,7 +86,7 @@ func (s *S3Store) GetByPrefix(ctx context.Context, prefix Key, keyOnly bool) ([]
objectsValues = append(objectsValues, value) objectsValues = append(objectsValues, value)
} }
} }
}else { } else {
return nil, nil, err 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)), Prefix: aws.String(string(keyStart)),
}) })
if err == nil && objects != nil { if err == nil && objects != nil {
for _, object := range objects.Contents{ for _, object := range objects.Contents {
if *object.Key >= string(keyEnd) { if *object.Key >= string(keyEnd) {
keys = append(keys, []byte(*object.Key)) keys = append(keys, []byte(*object.Key))
if !keyOnly { 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 { func (s *S3Store) Delete(ctx context.Context, key Key) error {
_, err := s.client.DeleteObjectWithContext(ctx, &s3.DeleteObjectInput{ _, err := s.client.DeleteObjectWithContext(ctx, &s3.DeleteObjectInput{
Bucket: aws.String(bucketName), Bucket: aws.String(bucketName),
Key: aws.String(string(key)), Key: aws.String(string(key)),
}) })
return err return err
} }
@ -142,7 +143,7 @@ func (s *S3Store) DeleteByPrefix(ctx context.Context, prefix Key) error {
for _, object := range objects.Contents { for _, object := range objects.Contents {
_, err := s.client.DeleteObjectWithContext(ctx, &s3.DeleteObjectInput{ _, err := s.client.DeleteObjectWithContext(ctx, &s3.DeleteObjectInput{
Bucket: aws.String(bucketName), Bucket: aws.String(bucketName),
Key: object.Key, Key: object.Key,
}) })
return err return err
} }
@ -160,10 +161,10 @@ func (s *S3Store) DeleteRange(ctx context.Context, keyStart Key, keyEnd Key) err
if objects != nil && err == nil { if objects != nil && err == nil {
for _, object := range objects.Contents { for _, object := range objects.Contents {
if *object.Key > string(keyEnd){ if *object.Key > string(keyEnd) {
_, err := s.client.DeleteObjectWithContext(ctx, &s3.DeleteObjectInput{ _, err := s.client.DeleteObjectWithContext(ctx, &s3.DeleteObjectInput{
Bucket: aws.String(bucketName), Bucket: aws.String(bucketName),
Key: object.Key, Key: object.Key,
}) })
return err return err
} }

View File

@ -2,6 +2,7 @@ package S3_driver
import ( import (
"context" "context"
"github.com/aws/aws-sdk-go/aws" "github.com/aws/aws-sdk-go/aws"
"github.com/aws/aws-sdk-go/aws/endpoints" "github.com/aws/aws-sdk-go/aws/endpoints"
"github.com/zilliztech/milvus-distributed/internal/storage/internal/minio/codec" "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) 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 return nil, err
} }
_, _, suffix, err := codec.MvccDecode(keys[0]) _, _, suffix, err := codec.MvccDecode(keys[0])
if err != nil{ if err != nil {
return nil, err return nil, err
} }
if suffix == "delete"{ if suffix == "delete" {
return nil, nil return nil, nil
} }
return values[0], err 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 var values []Value
for i, key := range keys{ for i, key := range keys {
value, err := s.GetRow(ctx, key, timestamps[i]) value, err := s.GetRow(ctx, key, timestamps[i])
if err!= nil{ if err != nil {
return nil, err return nil, err
} }
values = append(values, value) values = append(values, value)
@ -141,32 +142,32 @@ func (s *S3Driver) GetRows(ctx context.Context, keys []Key, timestamps []Timesta
return values, nil 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) minioKey, err := codec.MvccEncode(key, timestamp, segment)
if err != nil{ if err != nil {
return err return err
} }
err = s.driver.Put(ctx, minioKey, value) err = s.driver.Put(ctx, minioKey, value)
return err 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 maxThread := 100
batchSize := 1 batchSize := 1
keysLength := len(keys) keysLength := len(keys)
if keysLength / batchSize > maxThread { if keysLength/batchSize > maxThread {
batchSize = keysLength / maxThread batchSize = keysLength / maxThread
} }
batchNums := keysLength / batchSize batchNums := keysLength / batchSize
if keysLength % batchSize != 0 { if keysLength%batchSize != 0 {
batchNums = keysLength / batchSize + 1 batchNums = keysLength/batchSize + 1
} }
errCh := make(chan error) errCh := make(chan error)
f := func(ctx2 context.Context, keys2 []Key, values2 []Value, segments2 []string, timestamps2 []Timestamp) { 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]) err := s.PutRow(ctx2, keys2[i], values2[i], segments2[i], timestamps2[i])
errCh <- err 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++ { for i := 0; i < len(keys); i++ {
if err := <- errCh; err != nil { if err := <-errCh; err != nil {
return err return err
} }
} }
return nil 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, "") keyEnd, err := codec.MvccEncode(key, timestamp, "")
if err != nil{ if err != nil {
return nil, err 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 { if err != nil {
return nil, err return nil, err
} }
@ -219,9 +220,9 @@ func (s *S3Driver) GetSegments(ctx context.Context, key Key, timestamp Timestamp
return segments, err 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") minioKey, err := codec.MvccEncode(key, timestamp, "delete")
if err != nil{ if err != nil {
return err return err
} }
value := []byte("0") value := []byte("0")
@ -229,24 +230,24 @@ func (s *S3Driver) DeleteRow(ctx context.Context, key Key, timestamp Timestamp)
return err 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 maxThread := 100
batchSize := 1 batchSize := 1
keysLength := len(keys) keysLength := len(keys)
if keysLength / batchSize > maxThread { if keysLength/batchSize > maxThread {
batchSize = keysLength / maxThread batchSize = keysLength / maxThread
} }
batchNums := keysLength / batchSize batchNums := keysLength / batchSize
if keysLength % batchSize != 0 { if keysLength%batchSize != 0 {
batchNums = keysLength / batchSize + 1 batchNums = keysLength/batchSize + 1
} }
errCh := make(chan error) errCh := make(chan error)
f := func(ctx2 context.Context, keys2 []Key, timestamps2 []Timestamp) { 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]) err := s.DeleteRow(ctx2, keys2[i], timestamps2[i])
errCh <- err errCh <- err
} }
@ -263,14 +264,14 @@ func (s *S3Driver) DeleteRows(ctx context.Context, keys []Key, timestamps []Time
} }
for i := 0; i < len(keys); i++ { for i := 0; i < len(keys); i++ {
if err := <- errCh; err != nil { if err := <-errCh; err != nil {
return err return err
} }
} }
return nil 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) logKey := codec.LogEncode(key, timestamp, channel)
err := s.driver.Put(ctx, logKey, value) err := s.driver.Put(ctx, logKey, value)
return err return err
@ -283,12 +284,12 @@ func (s *S3Driver) GetLog(ctx context.Context, start Timestamp, end Timestamp, c
} }
var resultValues []Value var resultValues []Value
for i, key := range keys{ for i, key := range keys {
_, ts, channel, err := codec.LogDecode(string(key)) _, ts, channel, err := codec.LogDecode(string(key))
if err != nil { if err != nil {
return nil, err return nil, err
} }
if ts >= start && ts <= end { if ts >= start && ts <= end {
for j := 0; j < len(channels); j++ { for j := 0; j < len(channels); j++ {
if channel == channels[j] { if channel == channels[j] {
resultValues = append(resultValues, values[i]) resultValues = append(resultValues, values[i])
@ -300,32 +301,32 @@ func (s *S3Driver) GetLog(ctx context.Context, start Timestamp, end Timestamp, c
return resultValues, nil 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")) 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) 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")) 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")) 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) 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")) return s.driver.Delete(ctx, codec.SegmentEncode(segment, "DL"))
} }

View File

@ -46,7 +46,7 @@ func MvccDecode(key []byte) (string, uint64, string, error) {
func LogEncode(key []byte, ts uint64, channel int) []byte { func LogEncode(key []byte, ts uint64, channel int) []byte {
suffix := string(key) + "_" + fmt.Sprintf("%d", channel) suffix := string(key) + "_" + fmt.Sprintf("%d", channel)
logKey, err := MvccEncode([]byte("log"), ts, suffix) logKey, err := MvccEncode([]byte("log"), ts, suffix)
if err != nil{ if err != nil {
return nil return nil
} }
return logKey return logKey

View File

@ -2,11 +2,12 @@ package minio_driver
import ( import (
"context" "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/internal/minio/codec" . "github.com/zilliztech/milvus-distributed/internal/storage/internal/minio/codec"
. "github.com/zilliztech/milvus-distributed/internal/storage/type" . "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 { 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) 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 return nil, err
} }
_, _, suffix, err := MvccDecode(keys[0]) _, _, suffix, err := MvccDecode(keys[0])
if err != nil{ if err != nil {
return nil, err return nil, err
} }
if suffix == "delete"{ if suffix == "delete" {
return nil, nil return nil, nil
} }
return values[0], err 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 var values []Value
for i, key := range keys{ for i, key := range keys {
value, err := s.GetRow(ctx, key, timestamps[i]) value, err := s.GetRow(ctx, key, timestamps[i])
if err!= nil{ if err != nil {
return nil, err return nil, err
} }
values = append(values, value) values = append(values, value)
@ -161,32 +162,32 @@ func (s *minioDriver) GetRows(ctx context.Context, keys []Key, timestamps []Time
return values, nil 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) minioKey, err := MvccEncode(key, timestamp, segment)
if err != nil{ if err != nil {
return err return err
} }
err = s.driver.Put(ctx, minioKey, value) err = s.driver.Put(ctx, minioKey, value)
return err 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 maxThread := 100
batchSize := 1 batchSize := 1
keysLength := len(keys) keysLength := len(keys)
if keysLength / batchSize > maxThread { if keysLength/batchSize > maxThread {
batchSize = keysLength / maxThread batchSize = keysLength / maxThread
} }
batchNums := keysLength / batchSize batchNums := keysLength / batchSize
if keysLength % batchSize != 0 { if keysLength%batchSize != 0 {
batchNums = keysLength / batchSize + 1 batchNums = keysLength/batchSize + 1
} }
errCh := make(chan error) errCh := make(chan error)
f := func(ctx2 context.Context, keys2 []Key, values2 []Value, segments2 []string, timestamps2 []Timestamp) { 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]) err := s.PutRow(ctx2, keys2[i], values2[i], segments2[i], timestamps2[i])
errCh <- err 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++ { for i := 0; i < len(keys); i++ {
if err := <- errCh; err != nil { if err := <-errCh; err != nil {
return err return err
} }
} }
return nil 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, "") keyEnd, err := MvccEncode(key, timestamp, "")
if err != nil{ if err != nil {
return nil, err 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 { if err != nil {
return nil, err return nil, err
} }
@ -239,9 +240,9 @@ func (s *minioDriver) GetSegments(ctx context.Context, key Key, timestamp Timest
return segments, err 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") minioKey, err := MvccEncode(key, timestamp, "delete")
if err != nil{ if err != nil {
return err return err
} }
value := []byte("0") value := []byte("0")
@ -249,24 +250,24 @@ func (s *minioDriver) DeleteRow(ctx context.Context, key Key, timestamp Timestam
return err 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 maxThread := 100
batchSize := 1 batchSize := 1
keysLength := len(keys) keysLength := len(keys)
if keysLength / batchSize > maxThread { if keysLength/batchSize > maxThread {
batchSize = keysLength / maxThread batchSize = keysLength / maxThread
} }
batchNums := keysLength / batchSize batchNums := keysLength / batchSize
if keysLength % batchSize != 0 { if keysLength%batchSize != 0 {
batchNums = keysLength / batchSize + 1 batchNums = keysLength/batchSize + 1
} }
errCh := make(chan error) errCh := make(chan error)
f := func(ctx2 context.Context, keys2 []Key, timestamps2 []Timestamp) { 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]) err := s.DeleteRow(ctx2, keys2[i], timestamps2[i])
errCh <- err errCh <- err
} }
@ -283,14 +284,14 @@ func (s *minioDriver) DeleteRows(ctx context.Context, keys []Key, timestamps []T
} }
for i := 0; i < len(keys); i++ { for i := 0; i < len(keys); i++ {
if err := <- errCh; err != nil { if err := <-errCh; err != nil {
return err return err
} }
} }
return nil 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) logKey := LogEncode(key, timestamp, channel)
err := s.driver.Put(ctx, logKey, value) err := s.driver.Put(ctx, logKey, value)
return err return err
@ -303,12 +304,12 @@ func (s *minioDriver) GetLog(ctx context.Context, start Timestamp, end Timestamp
} }
var resultValues []Value var resultValues []Value
for i, key := range keys{ for i, key := range keys {
_, ts, channel, err := LogDecode(string(key)) _, ts, channel, err := LogDecode(string(key))
if err != nil { if err != nil {
return nil, err return nil, err
} }
if ts >= start && ts <= end { if ts >= start && ts <= end {
for j := 0; j < len(channels); j++ { for j := 0; j < len(channels); j++ {
if channel == channels[j] { if channel == channels[j] {
resultValues = append(resultValues, values[i]) resultValues = append(resultValues, values[i])
@ -320,32 +321,32 @@ func (s *minioDriver) GetLog(ctx context.Context, start Timestamp, end Timestamp
return resultValues, nil 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")) 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) 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")) 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")) 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) 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")) return s.driver.Delete(ctx, SegmentEncode(segment, "DL"))
} }

View File

@ -3,10 +3,11 @@ package minio_driver
import ( import (
"bytes" "bytes"
"context" "context"
"io"
"github.com/minio/minio-go/v7"
"github.com/zilliztech/milvus-distributed/internal/conf" "github.com/zilliztech/milvus-distributed/internal/conf"
. "github.com/zilliztech/milvus-distributed/internal/storage/type" . "github.com/zilliztech/milvus-distributed/internal/storage/type"
"github.com/minio/minio-go/v7"
"io"
) )
var bucketName = conf.Config.Writer.Bucket var bucketName = conf.Config.Writer.Bucket
@ -32,7 +33,7 @@ func (s *minioStore) Get(ctx context.Context, key Key) (Value, error) {
return nil, err return nil, err
} }
size := 256*1024 size := 256 * 1024
buf := make([]byte, size) buf := make([]byte, size)
n, err := object.Read(buf) n, err := object.Read(buf)
if err != nil && err != io.EOF { 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 { for object := range objects {
objectsKeys = append(objectsKeys, []byte(object.Key)) objectsKeys = append(objectsKeys, []byte(object.Key))
if !keyOnly{ if !keyOnly {
value, err := s.Get(ctx, []byte(object.Key)) value, err := s.Get(ctx, []byte(object.Key))
if err != nil{ if err != nil {
return nil, nil, err return nil, nil, err
} }
objectsValues = append(objectsValues, value) 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 keys []Key
var values []Value var values []Value
limitCount := uint(limit) 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) { if object.Key >= string(keyEnd) {
keys = append(keys, []byte(object.Key)) keys = append(keys, []byte(object.Key))
if !keyOnly { if !keyOnly {
@ -77,7 +78,7 @@ func (s *minioStore) Scan(ctx context.Context, keyStart Key, keyEnd Key, limit i
values = append(values, value) values = append(values, value)
} }
limitCount-- limitCount--
if limitCount <= 0{ if limitCount <= 0 {
break break
} }
} }
@ -91,18 +92,18 @@ func (s *minioStore) Delete(ctx context.Context, key Key) error {
return err 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) objectsCh := make(chan minio.ObjectInfo)
go func() { go func() {
defer close(objectsCh) 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 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 { if rErr.Err != nil {
return rErr.Err return rErr.Err
} }
@ -116,14 +117,14 @@ func (s *minioStore) DeleteRange(ctx context.Context, keyStart Key, keyEnd Key)
go func() { go func() {
defer close(objectsCh) 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) { if object.Key <= string(keyEnd) {
objectsCh <- object 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 { if rErr.Err != nil {
return rErr.Err return rErr.Err
} }

View File

@ -2,15 +2,15 @@ package minio_driver_test
import ( import (
"context" "context"
minio_driver "github.com/zilliztech/milvus-distributed/internal/storage/internal/minio"
"github.com/stretchr/testify/assert"
"testing" "testing"
"github.com/stretchr/testify/assert"
minio_driver "github.com/zilliztech/milvus-distributed/internal/storage/internal/minio"
) )
var ctx = context.Background() var ctx = context.Background()
var client, err = minio_driver.NewMinioDriver(ctx) var client, err = minio_driver.NewMinioDriver(ctx)
func TestMinioDriver_PutRowAndGetRow(t *testing.T) { func TestMinioDriver_PutRowAndGetRow(t *testing.T) {
err = client.PutRow(ctx, []byte("bar"), []byte("abcdefghijklmnoopqrstuvwxyz"), "SegmentA", 1) err = client.PutRow(ctx, []byte("bar"), []byte("abcdefghijklmnoopqrstuvwxyz"), "SegmentA", 1)
assert.Nil(t, err) assert.Nil(t, err)
@ -30,7 +30,7 @@ func TestMinioDriver_PutRowAndGetRow(t *testing.T) {
assert.Equal(t, "testkeybarorbar_1", string(object)) 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) err = client.DeleteRow(ctx, []byte("bar"), 5)
assert.Nil(t, err) assert.Nil(t, err)
object, _ := client.GetRow(ctx, []byte("bar"), 6) 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")} keys := [][]byte{[]byte("foo"), []byte("bar")}
values := [][]byte{[]byte("The key is foo!"), []byte("The key is bar!")} values := [][]byte{[]byte("The key is foo!"), []byte("The key is bar!")}
segments := []string{"segmentA", "segmentB"} segments := []string{"segmentA", "segmentB"}
@ -77,7 +77,7 @@ func TestMinioDriver_PutRowsAndGetRows(t *testing.T){
assert.Equal(t, "The key is bar!", string(objects[1])) 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")} keys := [][]byte{[]byte("foo"), []byte("bar")}
timestamps := []uint64{3, 3} timestamps := []uint64{3, 3}
err := client.DeleteRows(ctx, keys, timestamps) err := client.DeleteRows(ctx, keys, timestamps)
@ -92,11 +92,11 @@ func TestMinioDriver_DeleteRows(t *testing.T){
func TestMinioDriver_PutLogAndGetLog(t *testing.T) { func TestMinioDriver_PutLogAndGetLog(t *testing.T) {
err = client.PutLog(ctx, []byte("insert"), []byte("This is insert log!"), 1, 11) err = client.PutLog(ctx, []byte("insert"), []byte("This is insert log!"), 1, 11)
assert.Nil(t, err) 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) 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) 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) assert.Nil(t, err)
channels := []int{5, 8, 9, 10, 11, 12, 13} channels := []int{5, 8, 9, 10, 11, 12, 13}
@ -119,7 +119,7 @@ func TestMinioDriver_Segment(t *testing.T) {
assert.Nil(t, err) 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!")) err := client.PutSegmentDL(ctx, "segmentB", []byte("This is segmentB's delete log!"))
assert.Nil(t, err) assert.Nil(t, err)
@ -129,4 +129,4 @@ func TestMinioDriver_SegmentDL(t *testing.T){
err = client.DeleteSegmentDL(ctx, "segmentB") err = client.DeleteSegmentDL(ctx, "segmentB")
assert.Nil(t, err) assert.Nil(t, err)
} }

View File

@ -3,6 +3,7 @@ package codec
import ( import (
"encoding/binary" "encoding/binary"
"errors" "errors"
"github.com/tikv/client-go/codec" "github.com/tikv/client-go/codec"
) )

View File

@ -3,13 +3,14 @@ package tikv_driver
import ( import (
"context" "context"
"errors" "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/conf"
. "github.com/zilliztech/milvus-distributed/internal/storage/internal/tikv/codec" . "github.com/zilliztech/milvus-distributed/internal/storage/internal/tikv/codec"
. "github.com/zilliztech/milvus-distributed/internal/storage/type" . "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 { func keyAddOne(key Key) Key {
@ -366,7 +367,7 @@ func (s *TikvStore) GetSegments(ctx context.Context, key Key, timestamp Timestam
if err != nil { if err != nil {
panic("must no error") panic("must no error")
} }
if ts <= timestamp && segment != string(DeleteMark){ if ts <= timestamp && segment != string(DeleteMark) {
segmentsSet[segment] = true segmentsSet[segment] = true
} }
} }

View File

@ -4,14 +4,15 @@ import (
"bytes" "bytes"
"context" "context"
"fmt" "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" "math"
"os" "os"
"sort" "sort"
"testing" "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 //var store TikvStore

View File

@ -3,16 +3,17 @@ package storage
import ( import (
"context" "context"
"errors" "errors"
S3Driver "github.com/zilliztech/milvus-distributed/internal/storage/internal/S3" S3Driver "github.com/zilliztech/milvus-distributed/internal/storage/internal/S3"
minIODriver "github.com/zilliztech/milvus-distributed/internal/storage/internal/minio" minIODriver "github.com/zilliztech/milvus-distributed/internal/storage/internal/minio"
tikvDriver "github.com/zilliztech/milvus-distributed/internal/storage/internal/tikv" 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) { func NewStore(ctx context.Context, driver storagetype.DriverType) (storagetype.Store, error) {
var err error var err error
var store storagetype.Store var store storagetype.Store
switch driver{ switch driver {
case storagetype.TIKVDriver: case storagetype.TIKVDriver:
store, err = tikvDriver.NewTikvStore(ctx) store, err = tikvDriver.NewTikvStore(ctx)
if err != nil { if err != nil {
@ -27,7 +28,7 @@ func NewStore(ctx context.Context, driver storagetype.DriverType) (storagetype.S
} }
return store, nil return store, nil
case storagetype.S3DRIVER: case storagetype.S3DRIVER:
store , err = S3Driver.NewS3Driver(ctx) store, err = S3Driver.NewS3Driver(ctx)
if err != nil { if err != nil {
//panic(err.Error()) //panic(err.Error())
return nil, err return nil, err
@ -35,4 +36,4 @@ func NewStore(ctx context.Context, driver storagetype.DriverType) (storagetype.S
return store, nil return store, nil
} }
return nil, errors.New("unsupported driver") return nil, errors.New("unsupported driver")
} }

View File

@ -14,7 +14,7 @@ type SegmentDL = []byte
const ( const (
MinIODriver DriverType = "MinIO" MinIODriver DriverType = "MinIO"
TIKVDriver DriverType = "TIKV" TIKVDriver DriverType = "TIKV"
S3DRIVER DriverType = "S3" S3DRIVER DriverType = "S3"
) )
/* /*

View File

@ -12,10 +12,10 @@ import (
"time" "time"
"github.com/apache/pulsar-client-go/pulsar" "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/conf"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb" "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
pb "github.com/zilliztech/milvus-distributed/internal/proto/message" pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/golang/protobuf/proto"
) )
type InsertLog struct { type InsertLog struct {

View File

@ -8,9 +8,9 @@ import (
"time" "time"
"github.com/apache/pulsar-client-go/pulsar" "github.com/apache/pulsar-client-go/pulsar"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb" "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
pb "github.com/zilliztech/milvus-distributed/internal/proto/message" pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/golang/protobuf/proto"
) )
const ( const (
@ -52,15 +52,15 @@ func TestAlignTimeSync(t *testing.T) {
} }
ts := []*internalpb.TimeSyncMsg{ ts := []*internalpb.TimeSyncMsg{
{ {
PeerId: 1, PeerId: 1,
Timestamp: toTimestamp(5), Timestamp: toTimestamp(5),
}, },
{ {
PeerId: 3, PeerId: 3,
Timestamp: toTimestamp(15), Timestamp: toTimestamp(15),
}, },
{ {
PeerId: 2, PeerId: 2,
Timestamp: toTimestamp(20), Timestamp: toTimestamp(20),
}, },
} }
@ -83,15 +83,15 @@ func TestAlignTimeSync2(t *testing.T) {
} }
ts := []*internalpb.TimeSyncMsg{ ts := []*internalpb.TimeSyncMsg{
{ {
PeerId: 1, PeerId: 1,
Timestamp: toTimestamp(5), Timestamp: toTimestamp(5),
}, },
{ {
PeerId: 3, PeerId: 3,
Timestamp: toTimestamp(150), Timestamp: toTimestamp(150),
}, },
{ {
PeerId: 2, PeerId: 2,
Timestamp: toTimestamp(20), Timestamp: toTimestamp(20),
}, },
} }
@ -112,23 +112,23 @@ func TestAlignTimeSync3(t *testing.T) {
} }
ts := []*internalpb.TimeSyncMsg{ ts := []*internalpb.TimeSyncMsg{
{ {
PeerId: 1, PeerId: 1,
Timestamp: toTimestamp(5), Timestamp: toTimestamp(5),
}, },
{ {
PeerId: 1, PeerId: 1,
Timestamp: toTimestamp(5), Timestamp: toTimestamp(5),
}, },
{ {
PeerId: 1, PeerId: 1,
Timestamp: toTimestamp(5), Timestamp: toTimestamp(5),
}, },
{ {
PeerId: 3, PeerId: 3,
Timestamp: toTimestamp(15), Timestamp: toTimestamp(15),
}, },
{ {
PeerId: 2, PeerId: 2,
Timestamp: toTimestamp(20), Timestamp: toTimestamp(20),
}, },
} }
@ -150,15 +150,15 @@ func TestAlignTimeSync4(t *testing.T) {
} }
ts := []*internalpb.TimeSyncMsg{ ts := []*internalpb.TimeSyncMsg{
{ {
PeerId: 1, PeerId: 1,
Timestamp: toTimestamp(15), Timestamp: toTimestamp(15),
}, },
{ {
PeerId: 1, PeerId: 1,
Timestamp: toTimestamp(25), Timestamp: toTimestamp(25),
}, },
{ {
PeerId: 1, PeerId: 1,
Timestamp: toTimestamp(35), Timestamp: toTimestamp(35),
}, },
} }
@ -181,23 +181,23 @@ func TestAlignTimeSync5(t *testing.T) {
} }
ts := []*internalpb.TimeSyncMsg{ ts := []*internalpb.TimeSyncMsg{
{ {
PeerId: 1, PeerId: 1,
Timestamp: toTimestamp(5), Timestamp: toTimestamp(5),
}, },
{ {
PeerId: 1, PeerId: 1,
Timestamp: toTimestamp(5), Timestamp: toTimestamp(5),
}, },
{ {
PeerId: 1, PeerId: 1,
Timestamp: toTimestamp(5), Timestamp: toTimestamp(5),
}, },
{ {
PeerId: 3, PeerId: 3,
Timestamp: toTimestamp(15), Timestamp: toTimestamp(15),
}, },
{ {
PeerId: 3, PeerId: 3,
Timestamp: toTimestamp(20), Timestamp: toTimestamp(20),
}, },
} }

View File

@ -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 package etcdutil
import ( import (
@ -19,9 +6,9 @@ import (
"net/http" "net/http"
"time" "time"
"github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/gogo/protobuf/proto" "github.com/gogo/protobuf/proto"
"github.com/pingcap/log" "github.com/pingcap/log"
"github.com/zilliztech/milvus-distributed/internal/errors"
"go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/clientv3"
"go.etcd.io/etcd/etcdserver" "go.etcd.io/etcd/etcdserver"
"go.etcd.io/etcd/pkg/types" "go.etcd.io/etcd/pkg/types"

View File

@ -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 package etcdutil
import ( import (

View File

@ -2,9 +2,10 @@ package flowgraph
import ( import (
"context" "context"
"sync"
"github.com/zilliztech/milvus-distributed/internal/errors" "github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil" "github.com/zilliztech/milvus-distributed/internal/util/typeutil"
"sync"
) )
type Timestamp = typeutil.Timestamp type Timestamp = typeutil.Timestamp
@ -24,8 +25,8 @@ type TimeTickedFlowGraph struct {
func (fg *TimeTickedFlowGraph) AddNode(node *Node) { func (fg *TimeTickedFlowGraph) AddNode(node *Node) {
nodeName := (*node).Name() nodeName := (*node).Name()
nodeCtx := nodeCtx{ nodeCtx := nodeCtx{
node: node, node: node,
inputChannels: make([]chan *Msg, 0), inputChannels: make([]chan *Msg, 0),
downstreamInputChanIdx: make(map[string]int), downstreamInputChanIdx: make(map[string]int),
} }
fg.nodeCtx[nodeName] = &nodeCtx fg.nodeCtx[nodeName] = &nodeCtx

View File

@ -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 package tsoutil
import ( import (
@ -22,7 +9,7 @@ const (
logicalBits = (1 << physicalShiftBits) - 1 logicalBits = (1 << physicalShiftBits) - 1
) )
func ComposeTS(physical, logical int64) uint64{ func ComposeTS(physical, logical int64) uint64 {
return uint64((physical << physicalShiftBits) + logical) 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()) physicalTime := time.Unix(int64(physical/1000), int64(physical)%1000*time.Millisecond.Nanoseconds())
return physicalTime, logical return physicalTime, logical
} }

View File

@ -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 package typeutil
import ( import (
"encoding/binary" "encoding/binary"
"github.com/zilliztech/milvus-distributed/internal/errors" "github.com/zilliztech/milvus-distributed/internal/errors"
) )

View File

@ -2,8 +2,9 @@ package typeutil
import ( import (
"encoding/binary" "encoding/binary"
"github.com/spaolacci/murmur3"
"unsafe" "unsafe"
"github.com/spaolacci/murmur3"
) )
func Hash32Bytes(b []byte) (uint32, error) { func Hash32Bytes(b []byte) (uint32, error) {
@ -20,7 +21,6 @@ func Hash32Uint64(v uint64) (uint32, error) {
return Hash32Bytes(b) return Hash32Bytes(b)
} }
func Hash32Int64(v int64) (uint32, error) { func Hash32Int64(v int64) (uint32, error) {
return Hash32Uint64(uint64(v)) return Hash32Uint64(uint64(v))
} }

View File

@ -1,8 +1,10 @@
package typeutil package typeutil
import ( import (
"github.com/stretchr/testify/assert"
"testing" "testing"
"unsafe" "unsafe"
"github.com/stretchr/testify/assert"
) )
func TestUint64(t *testing.T) { func TestUint64(t *testing.T) {
@ -27,4 +29,3 @@ func TestHash32_Uint64(t *testing.T) {
t.Log(h2) t.Log(h2)
assert.Equal(t, h, h2) assert.Equal(t, h, h2)
} }

View File

@ -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 package typeutil
import "time" import "time"

View File

@ -1,5 +1,4 @@
package typeutil package typeutil
type Timestamp = uint64 type Timestamp = uint64
type Id = int64 type Id = int64

View File

@ -5,16 +5,17 @@ import (
"encoding/binary" "encoding/binary"
"encoding/json" "encoding/json"
"fmt" "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" "log"
"os" "os"
"strconv" "strconv"
"sync" "sync"
"time" "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 { type SegmentIdInfo struct {

View File

@ -2,10 +2,11 @@ package test
import ( import (
"context" "context"
msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/zilliztech/milvus-distributed/internal/writer"
"sync" "sync"
"testing" "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 { 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, CollectionName: collectionName,
PartitionTag: partitionTag, PartitionTag: partitionTag,
SegmentId: int64(entityId / 100), SegmentId: int64(entityId / 100),
Uid: int64(entityId), Uid: int64(entityId),
Timestamp: uint64(entityId), Timestamp: uint64(entityId),
ClientId: 0, ClientId: 0,
} }
@ -22,7 +23,7 @@ func GetInsertMsg(collectionName string, partitionTag string, entityId int64) *m
func GetDeleteMsg(collectionName string, entityId int64) *msgpb.InsertOrDeleteMsg { func GetDeleteMsg(collectionName string, entityId int64) *msgpb.InsertOrDeleteMsg {
return &msgpb.InsertOrDeleteMsg{ return &msgpb.InsertOrDeleteMsg{
CollectionName: collectionName, CollectionName: collectionName,
Uid: entityId, Uid: entityId,
Timestamp: uint64(entityId + 100), Timestamp: uint64(entityId + 100),
} }
} }

View File

@ -2,12 +2,13 @@ package test
import ( import (
"context" "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" "log"
"testing" "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) { func TestKey2Seg(t *testing.T) {