Add timestamp allocator

Signed-off-by: zhenshan.cao <zhenshan.cao@zilliz.com>
pull/4973/head^2
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"
)
func main() {
var yamlFile string

View File

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

View File

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

View File

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

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
import (
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
"context"
"fmt"
"log"
"time"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
)
type IdAllocator struct {
Allocator
idStart int64
idEnd int64
}
func NewIdAllocator(ctx context.Context) (*IdAllocator, error) {
ctx1, cancel := context.WithCancel(ctx)
a := &IdAllocator{
Allocator: Allocator{reqs: make(chan request, maxMergeRequests),
ctx: ctx1,
cancel: cancel,
},
}
a.tChan = &emptyTicker{}
a.Allocator.syncFunc = a.syncId
a.Allocator.processFunc = a.processFunc
return a, nil
}
func (ta *IdAllocator) syncId() {
fmt.Println("syncId")
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
req := &internalpb.IdRequest{
PeerId: 1,
Role: internalpb.PeerRole_Proxy,
Count: ta.countPerRpc,
}
resp, err := ta.masterClient.AllocId(ctx, req)
cancel()
if err != nil {
log.Panic("syncId Failed!!!!!")
return
}
ta.idStart = resp.GetId()
ta.idEnd = ta.idStart + int64(resp.GetCount())
}
func (allocator *IdAllocator) Initialize() error {
return nil
func (ta *IdAllocator) processFunc(req request) {
idRequest := req.(*idRequest)
idRequest.id = 1
fmt.Println("process Id")
}
func (allocator *IdAllocator) Start() error{
return nil
}
func (allocator *IdAllocator) Close() error{
return nil
func (ta *IdAllocator) AllocOne() (int64, error) {
ret, _, err := ta.Alloc(1)
if err != nil {
return 0, err
}
return ret, nil
}
func (allocator *IdAllocator) AllocOne() typeutil.Id {
return 1
func (ta *IdAllocator) Alloc(count uint32) (int64, int64, error) {
req := &idRequest{baseRequest: baseRequest{done: make(chan error), valid: false}}
req.count = count
ta.reqs <- req
req.Wait()
if !req.IsValid() {
return 0, 0, nil
}
start, count := int64(req.id), req.count
return start, start + int64(count), nil
}
func (allocator *IdAllocator) Alloc(count uint32) ([]typeutil.Id, error){
return make([]typeutil.Id, count), nil
}
func NewIdAllocator() *IdAllocator{
return &IdAllocator{}
}

View File

@ -1,32 +1,98 @@
package allocator
import (
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
"context"
"fmt"
"log"
"time"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
)
type TimestampAllocator struct {}
type Timestamp = uint64
func (allocator *TimestampAllocator) Start() error{
return nil
const (
tsCountPerRPC = 2 << 18 * 10
defaultUpdateInterval = 1000 * time.Millisecond
)
type TimestampAllocator struct {
Allocator
lastTsBegin uint64
lastTsEnd uint64
}
func (allocator *TimestampAllocator) Close() error{
return nil
func NewTimestampAllocator(ctx context.Context) (*TimestampAllocator, error) {
ctx1, cancel := context.WithCancel(ctx)
a := &TimestampAllocator{
Allocator: Allocator{reqs: make(chan request, maxMergeRequests),
ctx: ctx1,
cancel: cancel,
},
}
a.tChan = &ticker{
updateInterval: time.Second,
}
a.Allocator.syncFunc = a.syncTs
a.Allocator.processFunc = a.processFunc
return a, nil
}
func (allocator *TimestampAllocator) AllocOne() (typeutil.Timestamp, error){
ret, err := allocator.Alloc(1)
if err != nil{
return typeutil.ZeroTimestamp, err
func (ta *TimestampAllocator) syncTs() {
fmt.Println("sync TS")
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
req := &internalpb.TsoRequest{
PeerId: 1,
Role: internalpb.PeerRole_Proxy,
Count: ta.countPerRpc,
}
resp, err := ta.masterClient.AllocTimestamp(ctx, req)
cancel()
if err != nil {
log.Panic("syncId Failed!!!!!")
return
}
ta.lastTsBegin = resp.GetTimestamp()
ta.lastTsEnd = ta.lastTsBegin + uint64(resp.GetCount())
}
func (ta *TimestampAllocator) processFunc(req request) {
if req == nil {
fmt.Println("Occur nil!!!!")
return
}
tsoRequest := req.(*tsoRequest)
tsoRequest.timestamp = 1
fmt.Println("process tso")
}
func (ta *TimestampAllocator) AllocOne() (Timestamp, error) {
ret, err := ta.Alloc(1)
if err != nil {
return 0, err
}
return ret[0], nil
}
func (allocator *TimestampAllocator) Alloc(count uint32) ([]typeutil.Timestamp, error){
// to do lock and accuire more by grpc request
return make([]typeutil.Timestamp, count), nil
}
func (ta *TimestampAllocator) Alloc(count uint32) ([]Timestamp, error) {
//req := tsoReqPool.Get().(*tsoRequest)
req := &tsoRequest{
baseRequest: baseRequest{done: make(chan error), valid: false},
}
req.count = count
ta.reqs <- req
fmt.Println("YYYYY ", len(ta.reqs))
req.Wait()
func NewTimestampAllocator() *TimestampAllocator{
return &TimestampAllocator{}
if !req.IsValid() {
return nil, nil
}
start, count := req.timestamp, req.count
var ret []Timestamp
for i := uint32(0); i < count; i++ {
ret = append(ret, start+uint64(i))
}
return ret, nil
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -5,21 +5,20 @@ import (
"strconv"
"time"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/master/collection"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"github.com/zilliztech/milvus-distributed/internal/allocator"
"github.com/zilliztech/milvus-distributed/internal/kv"
"github.com/zilliztech/milvus-distributed/internal/master/segment"
)
"github.com/zilliztech/milvus-distributed/internal/master/id"
var IdAllocator *allocator.IdAllocator = allocator.NewIdAllocator()
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/kv"
"github.com/zilliztech/milvus-distributed/internal/master/collection"
"github.com/zilliztech/milvus-distributed/internal/master/segment"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
)
func CollectionController(ch chan *schemapb.CollectionSchema, kvbase kv.Base, errch chan error) {
for collectionMeta := range ch {
sID := IdAllocator.AllocOne()
cID := IdAllocator.AllocOne()
s2ID := IdAllocator.AllocOne()
sID, _ := id.AllocOne()
cID, _ := id.AllocOne()
s2ID, _ := id.AllocOne()
fieldMetas := []*schemapb.FieldSchema{}
if collectionMeta.Fields != nil {
fieldMetas = collectionMeta.Fields
@ -55,8 +54,8 @@ func CollectionController(ch chan *schemapb.CollectionSchema, kvbase kv.Base, er
}
func WriteCollection2Datastore(collectionMeta *schemapb.CollectionSchema, kvbase kv.Base) error {
sID := IdAllocator.AllocOne()
cID := IdAllocator.AllocOne()
sID, _ := id.AllocOne()
cID, _ := id.AllocOne()
fieldMetas := []*schemapb.FieldSchema{}
if collectionMeta.Fields != nil {
fieldMetas = collectionMeta.Fields

View File

@ -5,13 +5,14 @@ import (
"strconv"
"time"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/master/collection"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/kv"
"github.com/zilliztech/milvus-distributed/internal/master/segment"
)
"github.com/zilliztech/milvus-distributed/internal/master/id"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/kv"
"github.com/zilliztech/milvus-distributed/internal/master/collection"
"github.com/zilliztech/milvus-distributed/internal/master/segment"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
)
func ComputeCloseTime(ss internalpb.SegmentStatistics, kvbase kv.Base) error {
if int(ss.MemorySize) > int(conf.Config.Master.SegmentThreshole*0.8) {
@ -38,7 +39,7 @@ func ComputeCloseTime(ss internalpb.SegmentStatistics, kvbase kv.Base) error {
}
kvbase.Save("segment/"+strconv.Itoa(int(ss.SegmentId)), updateData)
//create new segment
newSegID := IdAllocator.AllocOne()
newSegID, _ := id.AllocOne()
newSeg := segment.NewSegment(newSegID, seg.CollectionID, seg.CollectionName, "default", seg.ChannelStart, seg.ChannelEnd, currentTime, time.Unix(1<<36-1, 0))
newSegData, err := segment.Segment2JSON(*&newSeg)
if err != nil {
@ -63,4 +64,3 @@ func ComputeCloseTime(ss internalpb.SegmentStatistics, kvbase kv.Base) error {
}
return nil
}

View File

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

View File

@ -2,17 +2,13 @@ package master
import (
"context"
"time"
"github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/master/id"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
"go.uber.org/zap"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
"io"
"log"
"time"
)
const slowThreshold = 5 * time.Millisecond
@ -340,38 +336,43 @@ func (s *Master) ShowPartitions(ctx context.Context, in *internalpb.ShowPartitio
return t.(*showPartitionTask).stringListResponse, nil
}
//----------------------------------------Internal GRPC Service--------------------------------
// Tso implements gRPC PDServer.
func (s *Master) Tso(stream masterpb.Master_TsoServer) error {
for {
request, err := stream.Recv()
if err == io.EOF {
return nil
}
if err != nil {
return errors.WithStack(err)
}
start := time.Now()
func (s *Master) AllocTimestamp(ctx context.Context, request *internalpb.TsoRequest) (*internalpb.TsoResponse, error) {
count := request.GetCount()
ts, err := s.tsoAllocator.GenerateTSO(count)
count := request.GetCount()
ts, err := s.tsoAllocator.GenerateTSO(count)
if err != nil {
return status.Errorf(codes.Unknown, err.Error())
}
elapsed := time.Since(start)
if elapsed > slowThreshold {
log.Println("get timestamp too slow", zap.Duration("cost", elapsed))
}
response := &internalpb.TsoResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS},
Timestamp: ts,
Count: count,
}
if err := stream.Send(response); err != nil {
return errors.WithStack(err)
}
if err != nil {
return &internalpb.TsoResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR},
}, err
}
}
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"
)
// GlobalTSOAllocator is the global single point TSO allocator.
type GlobalIdAllocator struct {
allocator tso.Allocator
}
func NewGlobalIdAllocator() *GlobalIdAllocator {
return &GlobalIdAllocator{
allocator: tso.NewGlobalTSOAllocator("idTimestamp"),
}
//func getID
var allocator GlobalIdAllocator = GlobalIdAllocator{
allocator: tso.NewGlobalTSOAllocator("idTimestamp"),
}
// Initialize will initialize the created global TSO allocator.
@ -36,9 +35,9 @@ func (gia *GlobalIdAllocator) Initialize() error {
// GenerateTSO is used to generate a given number of TSOs.
// Make sure you have initialized the TSO allocator before calling.
func (gia *GlobalIdAllocator) Generate(count uint32) (int64, int64, error) {
timestamp, err:= gia.allocator.GenerateTSO(count)
if err != nil{
func (gia *GlobalIdAllocator) Alloc(count uint32) (int64, int64, error) {
timestamp, err := gia.allocator.GenerateTSO(count)
if err != nil {
return 0, 0, err
}
idStart := int64(timestamp)
@ -46,3 +45,19 @@ func (gia *GlobalIdAllocator) Generate(count uint32) (int64, int64, error) {
return idStart, idEnd, nil
}
func (gia *GlobalIdAllocator) AllocOne() (int64, error) {
timestamp, err := gia.allocator.GenerateTSO(1)
if err != nil {
return 0, err
}
idStart := int64(timestamp)
return idStart, nil
}
func AllocOne() (int64, error) {
return allocator.AllocOne()
}
func Alloc(count uint32) (int64, int64, error) {
return allocator.Alloc(count)
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

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

View File

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

View File

@ -89,5 +89,6 @@ service Master {
rpc ShowPartitions(internal.ShowPartitionRequest) returns (service.StringListResponse) {}
rpc Tso(stream internal.TsoRequest) returns (stream internal.TsoResponse) {}
rpc AllocTimestamp(internal.TsoRequest) returns (internal.TsoResponse) {}
rpc AllocId(internal.IdRequest) returns (internal.IdResponse) {}
}

View File

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

View File

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

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

View File

@ -4,27 +4,35 @@ import (
"context"
"encoding/binary"
"encoding/json"
"github.com/apache/pulsar-client-go/pulsar"
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
"github.com/zilliztech/milvus-distributed/internal/allocator"
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/golang/protobuf/proto"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
etcd "go.etcd.io/etcd/clientv3"
"google.golang.org/grpc"
"os"
"sort"
"strconv"
"testing"
"time"
)
"github.com/apache/pulsar-client-go/pulsar"
"github.com/golang/protobuf/proto"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/allocator"
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
etcd "go.etcd.io/etcd/clientv3"
"google.golang.org/grpc"
)
const (
tsoKeyPath string = "/timestampOracle"
)
var timeAllocator *allocator.TimestampAllocator = allocator.NewTimestampAllocator()
var timeAllocator *allocator.TimestampAllocator
func TestMain(m *testing.M) {
timeAllocator, _ = allocator.NewTimestampAllocator(context.Background())
exitCode := m.Run()
timeAllocator.Close()
os.Exit(exitCode)
}
func TestProxyNode(t *testing.T) {
startTestMaster("localhost:11000", t)
@ -75,10 +83,10 @@ func TestProxyNode(t *testing.T) {
assert.Nil(t, err)
curValue, err := testOpt.tso.AllocOne()
curTS, err := timeAllocator.AllocOne()
curTS, err := timeAllocator.AllocOne()
assert.Equalf(t, err, nil, "%s", "allocator failed")
curTime, _:= tsoutil.ParseTS(curTS)
curTime, _ := tsoutil.ParseTS(curTS)
t.Logf("current time stamp = %d, saved time stamp = %d", curTime, value)
assert.GreaterOrEqual(t, curValue, value)
assert.GreaterOrEqual(t, value, startTime)
@ -163,7 +171,7 @@ func TestProxyNode(t *testing.T) {
if err := proto.Unmarshal(cm.Payload(), &tsm); err != nil {
t.Fatal(err)
}
curT, _:= tsoutil.ParseTS(tsm.Timestamp)
curT, _ := tsoutil.ParseTS(tsm.Timestamp)
t.Logf("time tick = %d", curT)
assert.Greater(t, curT, lastT)
lastT = curT
@ -252,7 +260,7 @@ func TestProxyNode(t *testing.T) {
physicalTime, _ := tsoutil.ParseTS(qm.Timestamp)
t.Logf("query time stamp = %d", physicalTime)
assert.Greater(t,physicalTime, startTime)
assert.Greater(t, physicalTime, startTime)
r1 := pb.QueryResult{
Status: &pb.Status{ErrorCode: pb.ErrorCode_SUCCESS},

View File

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

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
import (
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
"sync"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
)
type requestScheduler struct {

View File

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

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 (
"context"
"github.com/apache/pulsar-client-go/pulsar"
"github.com/zilliztech/milvus-distributed/internal/errors"
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
"log"
"time"
"github.com/apache/pulsar-client-go/pulsar"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/errors"
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
)
type timeTick struct {
@ -49,10 +50,10 @@ func (tt *timeTick) tick() error {
return nil
}
func (tt *timeTick) Restart() error{
func (tt *timeTick) Restart() error {
tt.lastTick = 0
ts, err := tt.getTimestamp()
if err != nil{
if err != nil {
return err
}

View File

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

View File

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

View File

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

View File

@ -3,15 +3,16 @@ package reader
import (
"context"
"encoding/binary"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"math"
"strconv"
"sync"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
)
func TestInsertAndDelete_MessagesPreprocess(t *testing.T) {

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -3,14 +3,15 @@ package reader
import (
"context"
"encoding/binary"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"math"
"strconv"
"sync"
"testing"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
)
// NOTE: start pulsar before test

View File

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

View File

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

View File

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

View File

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

View File

@ -2,13 +2,14 @@ package reader
import (
"context"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"strconv"
"testing"
"time"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/stretchr/testify/assert"
)

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -4,14 +4,15 @@ import (
"bytes"
"context"
"fmt"
. "github.com/zilliztech/milvus-distributed/internal/storage/internal/tikv/codec"
. "github.com/zilliztech/milvus-distributed/internal/storage/type"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"math"
"os"
"sort"
"testing"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
. "github.com/zilliztech/milvus-distributed/internal/storage/internal/tikv/codec"
. "github.com/zilliztech/milvus-distributed/internal/storage/type"
)
//var store TikvStore

View File

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

View File

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

View File

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

View File

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

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

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
import (

View File

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

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

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

View File

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

View File

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

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
import "time"

View File

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

View File

@ -5,16 +5,17 @@ import (
"encoding/binary"
"encoding/json"
"fmt"
"github.com/zilliztech/milvus-distributed/internal/conf"
msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/zilliztech/milvus-distributed/internal/storage"
storagetype "github.com/zilliztech/milvus-distributed/internal/storage/type"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
"log"
"os"
"strconv"
"sync"
"time"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/zilliztech/milvus-distributed/internal/storage"
storagetype "github.com/zilliztech/milvus-distributed/internal/storage/type"
)
type SegmentIdInfo struct {

View File

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

View File

@ -2,12 +2,13 @@ package test
import (
"context"
"github.com/apache/pulsar-client-go/pulsar"
msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/gogo/protobuf/proto"
"github.com/stretchr/testify/assert"
"log"
"testing"
"github.com/apache/pulsar-client-go/pulsar"
"github.com/gogo/protobuf/proto"
"github.com/stretchr/testify/assert"
msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message"
)
func TestKey2Seg(t *testing.T) {