mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-08 01:58:34 +08:00
Add timestamp allocator
Signed-off-by: zhenshan.cao <zhenshan.cao@zilliz.com>
This commit is contained in:
parent
d7ef21c644
commit
e962b57fbb
@ -12,7 +12,6 @@ import (
|
|||||||
"syscall"
|
"syscall"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|
||||||
func main() {
|
func main() {
|
||||||
|
|
||||||
var yamlFile string
|
var yamlFile string
|
||||||
|
|||||||
@ -25,4 +25,3 @@ func main() {
|
|||||||
pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
|
pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
|
||||||
reader.StartQueryNode(ctx, pulsarAddr)
|
reader.StartQueryNode(ctx, pulsarAddr)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@ -5,9 +5,9 @@ import (
|
|||||||
"crypto/md5"
|
"crypto/md5"
|
||||||
"flag"
|
"flag"
|
||||||
"fmt"
|
"fmt"
|
||||||
|
"github.com/pivotal-golang/bytefmt"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/storage"
|
"github.com/zilliztech/milvus-distributed/internal/storage"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/storage/type"
|
"github.com/zilliztech/milvus-distributed/internal/storage/type"
|
||||||
"github.com/pivotal-golang/bytefmt"
|
|
||||||
"log"
|
"log"
|
||||||
"math/rand"
|
"math/rand"
|
||||||
"os"
|
"os"
|
||||||
@ -31,7 +31,6 @@ var logFile *os.File
|
|||||||
var store storagetype.Store
|
var store storagetype.Store
|
||||||
var wg sync.WaitGroup
|
var wg sync.WaitGroup
|
||||||
|
|
||||||
|
|
||||||
func runSet() {
|
func runSet() {
|
||||||
for time.Now().Before(endTime) {
|
for time.Now().Before(endTime) {
|
||||||
num := atomic.AddInt32(&keyNum, 1)
|
num := atomic.AddInt32(&keyNum, 1)
|
||||||
@ -72,7 +71,6 @@ func runBatchSet() {
|
|||||||
wg.Done()
|
wg.Done()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
func runGet() {
|
func runGet() {
|
||||||
for time.Now().Before(endTime) {
|
for time.Now().Before(endTime) {
|
||||||
num := atomic.AddInt32(&counter, 1)
|
num := atomic.AddInt32(&counter, 1)
|
||||||
|
|||||||
@ -5,8 +5,8 @@ import (
|
|||||||
"flag"
|
"flag"
|
||||||
"fmt"
|
"fmt"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/storage"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/msgclient"
|
"github.com/zilliztech/milvus-distributed/internal/msgclient"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/storage"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/writer"
|
"github.com/zilliztech/milvus-distributed/internal/writer"
|
||||||
"log"
|
"log"
|
||||||
"strconv"
|
"strconv"
|
||||||
|
|||||||
207
internal/allocator/allocator.go
Normal file
207
internal/allocator/allocator.go
Normal 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"))
|
||||||
|
}
|
||||||
@ -1,35 +1,79 @@
|
|||||||
package allocator
|
package allocator
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
"context"
|
||||||
|
"fmt"
|
||||||
|
"log"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||||
)
|
)
|
||||||
|
|
||||||
type IdAllocator struct {
|
type IdAllocator struct {
|
||||||
|
Allocator
|
||||||
|
|
||||||
|
idStart int64
|
||||||
|
idEnd int64
|
||||||
|
}
|
||||||
|
|
||||||
|
func NewIdAllocator(ctx context.Context) (*IdAllocator, error) {
|
||||||
|
ctx1, cancel := context.WithCancel(ctx)
|
||||||
|
a := &IdAllocator{
|
||||||
|
Allocator: Allocator{reqs: make(chan request, maxMergeRequests),
|
||||||
|
ctx: ctx1,
|
||||||
|
cancel: cancel,
|
||||||
|
},
|
||||||
|
}
|
||||||
|
a.tChan = &emptyTicker{}
|
||||||
|
a.Allocator.syncFunc = a.syncId
|
||||||
|
a.Allocator.processFunc = a.processFunc
|
||||||
|
return a, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (ta *IdAllocator) syncId() {
|
||||||
|
fmt.Println("syncId")
|
||||||
|
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
|
||||||
|
req := &internalpb.IdRequest{
|
||||||
|
PeerId: 1,
|
||||||
|
Role: internalpb.PeerRole_Proxy,
|
||||||
|
Count: ta.countPerRpc,
|
||||||
|
}
|
||||||
|
resp, err := ta.masterClient.AllocId(ctx, req)
|
||||||
|
|
||||||
|
cancel()
|
||||||
|
if err != nil {
|
||||||
|
log.Panic("syncId Failed!!!!!")
|
||||||
|
return
|
||||||
|
}
|
||||||
|
ta.idStart = resp.GetId()
|
||||||
|
ta.idEnd = ta.idStart + int64(resp.GetCount())
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (allocator *IdAllocator) Initialize() error {
|
func (ta *IdAllocator) processFunc(req request) {
|
||||||
return nil
|
idRequest := req.(*idRequest)
|
||||||
|
idRequest.id = 1
|
||||||
|
fmt.Println("process Id")
|
||||||
}
|
}
|
||||||
|
|
||||||
func (allocator *IdAllocator) Start() error{
|
func (ta *IdAllocator) AllocOne() (int64, error) {
|
||||||
return nil
|
ret, _, err := ta.Alloc(1)
|
||||||
|
if err != nil {
|
||||||
|
return 0, err
|
||||||
}
|
}
|
||||||
func (allocator *IdAllocator) Close() error{
|
return ret, nil
|
||||||
return nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (allocator *IdAllocator) AllocOne() typeutil.Id {
|
func (ta *IdAllocator) Alloc(count uint32) (int64, int64, error) {
|
||||||
return 1
|
req := &idRequest{baseRequest: baseRequest{done: make(chan error), valid: false}}
|
||||||
|
|
||||||
|
req.count = count
|
||||||
|
ta.reqs <- req
|
||||||
|
req.Wait()
|
||||||
|
|
||||||
|
if !req.IsValid() {
|
||||||
|
return 0, 0, nil
|
||||||
}
|
}
|
||||||
|
start, count := int64(req.id), req.count
|
||||||
func (allocator *IdAllocator) Alloc(count uint32) ([]typeutil.Id, error){
|
return start, start + int64(count), nil
|
||||||
return make([]typeutil.Id, count), nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
func NewIdAllocator() *IdAllocator{
|
|
||||||
return &IdAllocator{}
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
|
|||||||
@ -1,32 +1,98 @@
|
|||||||
package allocator
|
package allocator
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
"context"
|
||||||
|
"fmt"
|
||||||
|
"log"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||||
)
|
)
|
||||||
|
|
||||||
type TimestampAllocator struct {}
|
type Timestamp = uint64
|
||||||
|
|
||||||
func (allocator *TimestampAllocator) Start() error{
|
const (
|
||||||
return nil
|
tsCountPerRPC = 2 << 18 * 10
|
||||||
|
defaultUpdateInterval = 1000 * time.Millisecond
|
||||||
|
)
|
||||||
|
|
||||||
|
type TimestampAllocator struct {
|
||||||
|
Allocator
|
||||||
|
lastTsBegin uint64
|
||||||
|
lastTsEnd uint64
|
||||||
}
|
}
|
||||||
|
|
||||||
func (allocator *TimestampAllocator) Close() error{
|
func NewTimestampAllocator(ctx context.Context) (*TimestampAllocator, error) {
|
||||||
return nil
|
ctx1, cancel := context.WithCancel(ctx)
|
||||||
|
a := &TimestampAllocator{
|
||||||
|
Allocator: Allocator{reqs: make(chan request, maxMergeRequests),
|
||||||
|
ctx: ctx1,
|
||||||
|
cancel: cancel,
|
||||||
|
},
|
||||||
|
}
|
||||||
|
a.tChan = &ticker{
|
||||||
|
updateInterval: time.Second,
|
||||||
|
}
|
||||||
|
a.Allocator.syncFunc = a.syncTs
|
||||||
|
a.Allocator.processFunc = a.processFunc
|
||||||
|
return a, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (allocator *TimestampAllocator) AllocOne() (typeutil.Timestamp, error){
|
func (ta *TimestampAllocator) syncTs() {
|
||||||
ret, err := allocator.Alloc(1)
|
fmt.Println("sync TS")
|
||||||
|
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 {
|
if err != nil {
|
||||||
return typeutil.ZeroTimestamp, err
|
log.Panic("syncId Failed!!!!!")
|
||||||
|
return
|
||||||
|
}
|
||||||
|
ta.lastTsBegin = resp.GetTimestamp()
|
||||||
|
ta.lastTsEnd = ta.lastTsBegin + uint64(resp.GetCount())
|
||||||
|
}
|
||||||
|
|
||||||
|
func (ta *TimestampAllocator) processFunc(req request) {
|
||||||
|
if req == nil {
|
||||||
|
fmt.Println("Occur nil!!!!")
|
||||||
|
return
|
||||||
|
}
|
||||||
|
tsoRequest := req.(*tsoRequest)
|
||||||
|
tsoRequest.timestamp = 1
|
||||||
|
fmt.Println("process tso")
|
||||||
|
}
|
||||||
|
|
||||||
|
func (ta *TimestampAllocator) AllocOne() (Timestamp, error) {
|
||||||
|
ret, err := ta.Alloc(1)
|
||||||
|
if err != nil {
|
||||||
|
return 0, err
|
||||||
}
|
}
|
||||||
return ret[0], nil
|
return ret[0], nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (allocator *TimestampAllocator) Alloc(count uint32) ([]typeutil.Timestamp, error){
|
func (ta *TimestampAllocator) Alloc(count uint32) ([]Timestamp, error) {
|
||||||
// to do lock and accuire more by grpc request
|
//req := tsoReqPool.Get().(*tsoRequest)
|
||||||
return make([]typeutil.Timestamp, count), nil
|
req := &tsoRequest{
|
||||||
|
baseRequest: baseRequest{done: make(chan error), valid: false},
|
||||||
|
}
|
||||||
|
req.count = count
|
||||||
|
ta.reqs <- req
|
||||||
|
fmt.Println("YYYYY ", len(ta.reqs))
|
||||||
|
req.Wait()
|
||||||
|
|
||||||
|
if !req.IsValid() {
|
||||||
|
return nil, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func NewTimestampAllocator() *TimestampAllocator{
|
start, count := req.timestamp, req.count
|
||||||
return &TimestampAllocator{}
|
var ret []Timestamp
|
||||||
|
for i := uint32(0); i < count; i++ {
|
||||||
|
ret = append(ret, start+uint64(i))
|
||||||
|
}
|
||||||
|
return ret, nil
|
||||||
}
|
}
|
||||||
|
|||||||
@ -8,4 +8,3 @@ import (
|
|||||||
func TestMain(m *testing.M) {
|
func TestMain(m *testing.M) {
|
||||||
fmt.Printf("Result: %v\n", Config)
|
fmt.Printf("Result: %v\n", Config)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@ -1,4 +1,3 @@
|
|||||||
|
|
||||||
package errors
|
package errors
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
|||||||
@ -2,11 +2,12 @@ package kv
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
|
||||||
"go.etcd.io/etcd/clientv3"
|
|
||||||
"log"
|
"log"
|
||||||
"path"
|
"path"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||||
|
"go.etcd.io/etcd/clientv3"
|
||||||
)
|
)
|
||||||
|
|
||||||
const (
|
const (
|
||||||
|
|||||||
@ -1,10 +1,11 @@
|
|||||||
package kv
|
package kv
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"github.com/stretchr/testify/assert"
|
|
||||||
"go.etcd.io/etcd/clientv3"
|
|
||||||
"path"
|
"path"
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
|
"github.com/stretchr/testify/assert"
|
||||||
|
"go.etcd.io/etcd/clientv3"
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestEtcdKV_Load(t *testing.T) {
|
func TestEtcdKV_Load(t *testing.T) {
|
||||||
|
|||||||
@ -3,10 +3,10 @@ package collection
|
|||||||
import (
|
import (
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
|
||||||
"github.com/gogo/protobuf/proto"
|
"github.com/gogo/protobuf/proto"
|
||||||
jsoniter "github.com/json-iterator/go"
|
jsoniter "github.com/json-iterator/go"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
||||||
)
|
)
|
||||||
|
|
||||||
var json = jsoniter.ConfigCompatibleWithStandardLibrary
|
var json = jsoniter.ConfigCompatibleWithStandardLibrary
|
||||||
|
|||||||
@ -3,8 +3,9 @@ package collection
|
|||||||
import (
|
import (
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
||||||
)
|
)
|
||||||
|
|
||||||
var (
|
var (
|
||||||
|
|||||||
@ -3,13 +3,14 @@ package master
|
|||||||
import (
|
import (
|
||||||
"encoding/json"
|
"encoding/json"
|
||||||
"errors"
|
"errors"
|
||||||
|
"log"
|
||||||
|
"strconv"
|
||||||
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
|
||||||
"log"
|
|
||||||
"strconv"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
const collectionMetaPrefix = "collection/"
|
const collectionMetaPrefix = "collection/"
|
||||||
|
|||||||
@ -5,21 +5,20 @@ import (
|
|||||||
"strconv"
|
"strconv"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
"github.com/zilliztech/milvus-distributed/internal/master/id"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/master/collection"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/allocator"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/kv"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/master/segment"
|
|
||||||
)
|
|
||||||
|
|
||||||
var IdAllocator *allocator.IdAllocator = allocator.NewIdAllocator()
|
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/kv"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/master/collection"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/master/segment"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
||||||
|
)
|
||||||
|
|
||||||
func CollectionController(ch chan *schemapb.CollectionSchema, kvbase kv.Base, errch chan error) {
|
func CollectionController(ch chan *schemapb.CollectionSchema, kvbase kv.Base, errch chan error) {
|
||||||
for collectionMeta := range ch {
|
for collectionMeta := range ch {
|
||||||
sID := IdAllocator.AllocOne()
|
sID, _ := id.AllocOne()
|
||||||
cID := IdAllocator.AllocOne()
|
cID, _ := id.AllocOne()
|
||||||
s2ID := IdAllocator.AllocOne()
|
s2ID, _ := id.AllocOne()
|
||||||
fieldMetas := []*schemapb.FieldSchema{}
|
fieldMetas := []*schemapb.FieldSchema{}
|
||||||
if collectionMeta.Fields != nil {
|
if collectionMeta.Fields != nil {
|
||||||
fieldMetas = collectionMeta.Fields
|
fieldMetas = collectionMeta.Fields
|
||||||
@ -55,8 +54,8 @@ func CollectionController(ch chan *schemapb.CollectionSchema, kvbase kv.Base, er
|
|||||||
}
|
}
|
||||||
|
|
||||||
func WriteCollection2Datastore(collectionMeta *schemapb.CollectionSchema, kvbase kv.Base) error {
|
func WriteCollection2Datastore(collectionMeta *schemapb.CollectionSchema, kvbase kv.Base) error {
|
||||||
sID := IdAllocator.AllocOne()
|
sID, _ := id.AllocOne()
|
||||||
cID := IdAllocator.AllocOne()
|
cID, _ := id.AllocOne()
|
||||||
fieldMetas := []*schemapb.FieldSchema{}
|
fieldMetas := []*schemapb.FieldSchema{}
|
||||||
if collectionMeta.Fields != nil {
|
if collectionMeta.Fields != nil {
|
||||||
fieldMetas = collectionMeta.Fields
|
fieldMetas = collectionMeta.Fields
|
||||||
|
|||||||
@ -5,13 +5,14 @@ import (
|
|||||||
"strconv"
|
"strconv"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
"github.com/zilliztech/milvus-distributed/internal/master/id"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/master/collection"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/kv"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/master/segment"
|
|
||||||
)
|
|
||||||
|
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/kv"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/master/collection"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/master/segment"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||||
|
)
|
||||||
|
|
||||||
func ComputeCloseTime(ss internalpb.SegmentStatistics, kvbase kv.Base) error {
|
func ComputeCloseTime(ss internalpb.SegmentStatistics, kvbase kv.Base) error {
|
||||||
if int(ss.MemorySize) > int(conf.Config.Master.SegmentThreshole*0.8) {
|
if int(ss.MemorySize) > int(conf.Config.Master.SegmentThreshole*0.8) {
|
||||||
@ -38,7 +39,7 @@ func ComputeCloseTime(ss internalpb.SegmentStatistics, kvbase kv.Base) error {
|
|||||||
}
|
}
|
||||||
kvbase.Save("segment/"+strconv.Itoa(int(ss.SegmentId)), updateData)
|
kvbase.Save("segment/"+strconv.Itoa(int(ss.SegmentId)), updateData)
|
||||||
//create new segment
|
//create new segment
|
||||||
newSegID := IdAllocator.AllocOne()
|
newSegID, _ := id.AllocOne()
|
||||||
newSeg := segment.NewSegment(newSegID, seg.CollectionID, seg.CollectionName, "default", seg.ChannelStart, seg.ChannelEnd, currentTime, time.Unix(1<<36-1, 0))
|
newSeg := segment.NewSegment(newSegID, seg.CollectionID, seg.CollectionName, "default", seg.ChannelStart, seg.ChannelEnd, currentTime, time.Unix(1<<36-1, 0))
|
||||||
newSegData, err := segment.Segment2JSON(*&newSeg)
|
newSegData, err := segment.Segment2JSON(*&newSeg)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -63,4 +64,3 @@ func ComputeCloseTime(ss internalpb.SegmentStatistics, kvbase kv.Base) error {
|
|||||||
}
|
}
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@ -2,17 +2,13 @@ package master
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
"time"
|
||||||
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/master/id"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
|
||||||
"go.uber.org/zap"
|
|
||||||
"google.golang.org/grpc/codes"
|
|
||||||
"google.golang.org/grpc/status"
|
|
||||||
"io"
|
|
||||||
"log"
|
|
||||||
"time"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
const slowThreshold = 5 * time.Millisecond
|
const slowThreshold = 5 * time.Millisecond
|
||||||
@ -340,38 +336,43 @@ func (s *Master) ShowPartitions(ctx context.Context, in *internalpb.ShowPartitio
|
|||||||
return t.(*showPartitionTask).stringListResponse, nil
|
return t.(*showPartitionTask).stringListResponse, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
//----------------------------------------Internal GRPC Service--------------------------------
|
//----------------------------------------Internal GRPC Service--------------------------------
|
||||||
|
|
||||||
// Tso implements gRPC PDServer.
|
func (s *Master) AllocTimestamp(ctx context.Context, request *internalpb.TsoRequest) (*internalpb.TsoResponse, error) {
|
||||||
func (s *Master) Tso(stream masterpb.Master_TsoServer) error {
|
|
||||||
for {
|
|
||||||
request, err := stream.Recv()
|
|
||||||
if err == io.EOF {
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
if err != nil {
|
|
||||||
return errors.WithStack(err)
|
|
||||||
}
|
|
||||||
start := time.Now()
|
|
||||||
|
|
||||||
count := request.GetCount()
|
count := request.GetCount()
|
||||||
ts, err := s.tsoAllocator.GenerateTSO(count)
|
ts, err := s.tsoAllocator.GenerateTSO(count)
|
||||||
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return status.Errorf(codes.Unknown, err.Error())
|
return &internalpb.TsoResponse{
|
||||||
|
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR},
|
||||||
|
}, err
|
||||||
}
|
}
|
||||||
|
|
||||||
elapsed := time.Since(start)
|
|
||||||
if elapsed > slowThreshold {
|
|
||||||
log.Println("get timestamp too slow", zap.Duration("cost", elapsed))
|
|
||||||
}
|
|
||||||
response := &internalpb.TsoResponse{
|
response := &internalpb.TsoResponse{
|
||||||
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS},
|
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR},
|
||||||
Timestamp: ts,
|
Timestamp: ts,
|
||||||
Count: count,
|
Count: count,
|
||||||
}
|
}
|
||||||
if err := stream.Send(response); err != nil {
|
|
||||||
return errors.WithStack(err)
|
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
|
||||||
}
|
}
|
||||||
@ -17,17 +17,16 @@ import (
|
|||||||
"github.com/zilliztech/milvus-distributed/internal/master/tso"
|
"github.com/zilliztech/milvus-distributed/internal/master/tso"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|
||||||
// GlobalTSOAllocator is the global single point TSO allocator.
|
// GlobalTSOAllocator is the global single point TSO allocator.
|
||||||
type GlobalIdAllocator struct {
|
type GlobalIdAllocator struct {
|
||||||
allocator tso.Allocator
|
allocator tso.Allocator
|
||||||
}
|
}
|
||||||
|
|
||||||
func NewGlobalIdAllocator() *GlobalIdAllocator {
|
//func getID
|
||||||
return &GlobalIdAllocator{
|
|
||||||
|
var allocator GlobalIdAllocator = GlobalIdAllocator{
|
||||||
allocator: tso.NewGlobalTSOAllocator("idTimestamp"),
|
allocator: tso.NewGlobalTSOAllocator("idTimestamp"),
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
// Initialize will initialize the created global TSO allocator.
|
// Initialize will initialize the created global TSO allocator.
|
||||||
func (gia *GlobalIdAllocator) Initialize() error {
|
func (gia *GlobalIdAllocator) Initialize() error {
|
||||||
@ -36,7 +35,7 @@ func (gia *GlobalIdAllocator) Initialize() error {
|
|||||||
|
|
||||||
// GenerateTSO is used to generate a given number of TSOs.
|
// GenerateTSO is used to generate a given number of TSOs.
|
||||||
// Make sure you have initialized the TSO allocator before calling.
|
// Make sure you have initialized the TSO allocator before calling.
|
||||||
func (gia *GlobalIdAllocator) Generate(count uint32) (int64, int64, error) {
|
func (gia *GlobalIdAllocator) Alloc(count uint32) (int64, int64, error) {
|
||||||
timestamp, err := gia.allocator.GenerateTSO(count)
|
timestamp, err := gia.allocator.GenerateTSO(count)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return 0, 0, err
|
return 0, 0, err
|
||||||
@ -46,3 +45,19 @@ func (gia *GlobalIdAllocator) Generate(count uint32) (int64, int64, error) {
|
|||||||
return idStart, idEnd, nil
|
return idStart, idEnd, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (gia *GlobalIdAllocator) AllocOne() (int64, error) {
|
||||||
|
timestamp, err := gia.allocator.GenerateTSO(1)
|
||||||
|
if err != nil {
|
||||||
|
return 0, err
|
||||||
|
}
|
||||||
|
idStart := int64(timestamp)
|
||||||
|
return idStart, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func AllocOne() (int64, error) {
|
||||||
|
return allocator.AllocOne()
|
||||||
|
}
|
||||||
|
|
||||||
|
func Alloc(count uint32) (int64, int64, error) {
|
||||||
|
return allocator.Alloc(count)
|
||||||
|
}
|
||||||
|
|||||||
@ -3,6 +3,14 @@ package master
|
|||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"fmt"
|
"fmt"
|
||||||
|
"log"
|
||||||
|
"math/rand"
|
||||||
|
"net"
|
||||||
|
"strconv"
|
||||||
|
"sync"
|
||||||
|
"sync/atomic"
|
||||||
|
"time"
|
||||||
|
|
||||||
"github.com/apache/pulsar-client-go/pulsar"
|
"github.com/apache/pulsar-client-go/pulsar"
|
||||||
"github.com/golang/protobuf/proto"
|
"github.com/golang/protobuf/proto"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||||
@ -12,13 +20,6 @@ import (
|
|||||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
"log"
|
|
||||||
"math/rand"
|
|
||||||
"net"
|
|
||||||
"strconv"
|
|
||||||
"sync"
|
|
||||||
"sync/atomic"
|
|
||||||
"time"
|
|
||||||
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/master/tso"
|
"github.com/zilliztech/milvus-distributed/internal/master/tso"
|
||||||
"go.etcd.io/etcd/clientv3"
|
"go.etcd.io/etcd/clientv3"
|
||||||
|
|||||||
@ -1,12 +1,13 @@
|
|||||||
package master
|
package master
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"strconv"
|
||||||
|
"sync"
|
||||||
|
|
||||||
"github.com/golang/protobuf/proto"
|
"github.com/golang/protobuf/proto"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/kv"
|
"github.com/zilliztech/milvus-distributed/internal/kv"
|
||||||
pb "github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
|
pb "github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
|
||||||
"strconv"
|
|
||||||
"sync"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
type metaTable struct {
|
type metaTable struct {
|
||||||
|
|||||||
@ -1,12 +1,13 @@
|
|||||||
package master
|
package master
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"testing"
|
||||||
|
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/kv"
|
"github.com/zilliztech/milvus-distributed/internal/kv"
|
||||||
pb "github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
|
pb "github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
||||||
"go.etcd.io/etcd/clientv3"
|
"go.etcd.io/etcd/clientv3"
|
||||||
"testing"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestMetaTable_DeletePartition(t *testing.T) {
|
func TestMetaTable_DeletePartition(t *testing.T) {
|
||||||
|
|||||||
@ -3,11 +3,12 @@ package master
|
|||||||
import (
|
import (
|
||||||
"encoding/json"
|
"encoding/json"
|
||||||
"errors"
|
"errors"
|
||||||
|
"log"
|
||||||
|
"strconv"
|
||||||
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
|
||||||
"log"
|
|
||||||
"strconv"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
const partitionMetaPrefix = "partition/"
|
const partitionMetaPrefix = "partition/"
|
||||||
|
|||||||
@ -2,6 +2,7 @@ package master
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/kv"
|
"github.com/zilliztech/milvus-distributed/internal/kv"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||||
|
|||||||
@ -2,13 +2,14 @@ package timesync
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
|
||||||
"log"
|
"log"
|
||||||
"sort"
|
"sort"
|
||||||
"strconv"
|
"strconv"
|
||||||
"sync"
|
"sync"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||||
|
|
||||||
"github.com/apache/pulsar-client-go/pulsar"
|
"github.com/apache/pulsar-client-go/pulsar"
|
||||||
"github.com/golang/protobuf/proto"
|
"github.com/golang/protobuf/proto"
|
||||||
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||||
@ -113,7 +114,6 @@ func (r *TimeTickReader) sendEOFMsg(ctx context.Context, msg *pulsar.ProducerMes
|
|||||||
wg.Done()
|
wg.Done()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
func TimeTickService() {
|
func TimeTickService() {
|
||||||
timeTickTopic := "timeTick"
|
timeTickTopic := "timeTick"
|
||||||
timeTickSubName := "master"
|
timeTickSubName := "master"
|
||||||
|
|||||||
@ -14,13 +14,14 @@
|
|||||||
package tso
|
package tso
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
|
|
||||||
"go.etcd.io/etcd/clientv3"
|
|
||||||
"strconv"
|
"strconv"
|
||||||
"sync/atomic"
|
"sync/atomic"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
|
||||||
|
"go.etcd.io/etcd/clientv3"
|
||||||
|
|
||||||
"github.com/pingcap/log"
|
"github.com/pingcap/log"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||||
|
|||||||
@ -14,13 +14,14 @@
|
|||||||
package tso
|
package tso
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"go.uber.org/zap"
|
|
||||||
"log"
|
"log"
|
||||||
"path"
|
"path"
|
||||||
"sync/atomic"
|
"sync/atomic"
|
||||||
"time"
|
"time"
|
||||||
"unsafe"
|
"unsafe"
|
||||||
|
|
||||||
|
"go.uber.org/zap"
|
||||||
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/util/etcdutil"
|
"github.com/zilliztech/milvus-distributed/internal/util/etcdutil"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
|
"github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
|
||||||
|
|||||||
@ -7,11 +7,11 @@ import (
|
|||||||
"strconv"
|
"strconv"
|
||||||
|
|
||||||
"github.com/apache/pulsar-client-go/pulsar"
|
"github.com/apache/pulsar-client-go/pulsar"
|
||||||
|
"github.com/golang/protobuf/proto"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||||
msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/timesync"
|
"github.com/zilliztech/milvus-distributed/internal/timesync"
|
||||||
"github.com/golang/protobuf/proto"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
type MessageType int
|
type MessageType int
|
||||||
|
|||||||
@ -2,17 +2,17 @@ package msgstream
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"github.com/apache/pulsar-client-go/pulsar"
|
|
||||||
commonPb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
|
||||||
"log"
|
"log"
|
||||||
"sync"
|
"sync"
|
||||||
|
|
||||||
|
"github.com/apache/pulsar-client-go/pulsar"
|
||||||
|
commonPb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||||
)
|
)
|
||||||
|
|
||||||
type Timestamp uint64
|
|
||||||
|
|
||||||
type MsgPack struct {
|
type MsgPack struct {
|
||||||
BeginTs Timestamp
|
BeginTs typeutil.Timestamp
|
||||||
EndTs Timestamp
|
EndTs typeutil.Timestamp
|
||||||
Msgs []*TsMsg
|
Msgs []*TsMsg
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -110,7 +110,6 @@ func (ms *PulsarMsgStream) InitMsgPackBuf(msgPackBufSize int64) {
|
|||||||
ms.receiveBuf = make(chan *MsgPack, msgPackBufSize)
|
ms.receiveBuf = make(chan *MsgPack, msgPackBufSize)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
func (ms *PulsarMsgStream) Produce(msgPack *MsgPack) commonPb.Status {
|
func (ms *PulsarMsgStream) Produce(msgPack *MsgPack) commonPb.Status {
|
||||||
tsMsgs := msgPack.Msgs
|
tsMsgs := msgPack.Msgs
|
||||||
if len(tsMsgs) <= 0 {
|
if len(tsMsgs) <= 0 {
|
||||||
@ -215,7 +214,7 @@ type PulsarTtMsgStream struct {
|
|||||||
inputBuf []*TsMsg
|
inputBuf []*TsMsg
|
||||||
unsolvedBuf []*TsMsg
|
unsolvedBuf []*TsMsg
|
||||||
msgPacks []*MsgPack
|
msgPacks []*MsgPack
|
||||||
lastTimeStamp Timestamp
|
lastTimeStamp typeutil.Timestamp
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ms *PulsarTtMsgStream) Start() {
|
func (ms *PulsarTtMsgStream) Start() {
|
||||||
@ -225,7 +224,7 @@ func (ms *PulsarTtMsgStream) Start(){
|
|||||||
func (ms *PulsarTtMsgStream) bufMsgPackToChannel() {
|
func (ms *PulsarTtMsgStream) bufMsgPackToChannel() {
|
||||||
wg := sync.WaitGroup{}
|
wg := sync.WaitGroup{}
|
||||||
wg.Add(len(ms.consumers))
|
wg.Add(len(ms.consumers))
|
||||||
eofMsgTimeStamp := make(map[int]Timestamp)
|
eofMsgTimeStamp := make(map[int]typeutil.Timestamp)
|
||||||
mu := sync.Mutex{}
|
mu := sync.Mutex{}
|
||||||
for i := 0; i < len(ms.consumers); i++ {
|
for i := 0; i < len(ms.consumers); i++ {
|
||||||
go ms.findTimeTick(context.Background(), i, eofMsgTimeStamp, &wg, &mu)
|
go ms.findTimeTick(context.Background(), i, eofMsgTimeStamp, &wg, &mu)
|
||||||
@ -259,7 +258,7 @@ func (ms *PulsarTtMsgStream) bufMsgPackToChannel() {
|
|||||||
|
|
||||||
func (ms *PulsarTtMsgStream) findTimeTick(ctx context.Context,
|
func (ms *PulsarTtMsgStream) findTimeTick(ctx context.Context,
|
||||||
channelIndex int,
|
channelIndex int,
|
||||||
eofMsgMap map[int]Timestamp,
|
eofMsgMap map[int]typeutil.Timestamp,
|
||||||
wg *sync.WaitGroup,
|
wg *sync.WaitGroup,
|
||||||
mu *sync.Mutex) {
|
mu *sync.Mutex) {
|
||||||
for {
|
for {
|
||||||
@ -289,8 +288,8 @@ func (ms *PulsarTtMsgStream) findTimeTick(ctx context.Context,
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func checkTimeTickMsg(msg map[int]Timestamp) (Timestamp, bool) {
|
func checkTimeTickMsg(msg map[int]typeutil.Timestamp) (typeutil.Timestamp, bool) {
|
||||||
checkMap := make(map[Timestamp]int)
|
checkMap := make(map[typeutil.Timestamp]int)
|
||||||
for _, v := range msg {
|
for _, v := range msg {
|
||||||
checkMap[v] += 1
|
checkMap[v] += 1
|
||||||
}
|
}
|
||||||
|
|||||||
@ -2,9 +2,10 @@ package msgstream
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
|
"testing"
|
||||||
|
|
||||||
commonPb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
commonPb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||||
internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||||
"testing"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
func repackFunc(msgs []*TsMsg, hashKeys [][]int32) map[int32]*MsgPack {
|
func repackFunc(msgs []*TsMsg, hashKeys [][]int32) map[int32]*MsgPack {
|
||||||
|
|||||||
@ -2,8 +2,9 @@ package msgstream
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
|
internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestNewStream_Insert(t *testing.T) {
|
func TestNewStream_Insert(t *testing.T) {
|
||||||
|
|||||||
@ -2,6 +2,7 @@ package msgstream
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||||
|
. "github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||||
)
|
)
|
||||||
|
|
||||||
type MsgType uint32
|
type MsgType uint32
|
||||||
@ -208,7 +209,6 @@ func (tst TimeSyncTask) HashKeys() []int32 {
|
|||||||
return tst.HashValues
|
return tst.HashValues
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
///////////////////////////////////////////Key2Seg//////////////////////////////////////////
|
///////////////////////////////////////////Key2Seg//////////////////////////////////////////
|
||||||
//type Key2SegTask struct {
|
//type Key2SegTask struct {
|
||||||
// internalPb.Key2SegMsg
|
// internalPb.Key2SegMsg
|
||||||
|
|||||||
@ -46,6 +46,18 @@ enum PeerRole {
|
|||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
message IdRequest {
|
||||||
|
int64 peer_id = 1;
|
||||||
|
PeerRole role = 2;
|
||||||
|
uint32 count = 3;
|
||||||
|
}
|
||||||
|
|
||||||
|
message IdResponse {
|
||||||
|
common.Status status = 1;
|
||||||
|
int64 id = 2;
|
||||||
|
uint32 count = 3;
|
||||||
|
}
|
||||||
|
|
||||||
message TsoRequest {
|
message TsoRequest {
|
||||||
int64 peer_id = 1;
|
int64 peer_id = 1;
|
||||||
PeerRole role = 2;
|
PeerRole role = 2;
|
||||||
|
|||||||
@ -122,6 +122,116 @@ func (PeerRole) EnumDescriptor() ([]byte, []int) {
|
|||||||
return fileDescriptor_7eb37f6b80b23116, []int{1}
|
return fileDescriptor_7eb37f6b80b23116, []int{1}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
type IdRequest struct {
|
||||||
|
PeerId int64 `protobuf:"varint,1,opt,name=peer_id,json=peerId,proto3" json:"peer_id,omitempty"`
|
||||||
|
Role PeerRole `protobuf:"varint,2,opt,name=role,proto3,enum=milvus.proto.internal.PeerRole" json:"role,omitempty"`
|
||||||
|
Count uint32 `protobuf:"varint,3,opt,name=count,proto3" json:"count,omitempty"`
|
||||||
|
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||||
|
XXX_unrecognized []byte `json:"-"`
|
||||||
|
XXX_sizecache int32 `json:"-"`
|
||||||
|
}
|
||||||
|
|
||||||
|
func (m *IdRequest) Reset() { *m = IdRequest{} }
|
||||||
|
func (m *IdRequest) String() string { return proto.CompactTextString(m) }
|
||||||
|
func (*IdRequest) ProtoMessage() {}
|
||||||
|
func (*IdRequest) Descriptor() ([]byte, []int) {
|
||||||
|
return fileDescriptor_7eb37f6b80b23116, []int{0}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (m *IdRequest) XXX_Unmarshal(b []byte) error {
|
||||||
|
return xxx_messageInfo_IdRequest.Unmarshal(m, b)
|
||||||
|
}
|
||||||
|
func (m *IdRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||||
|
return xxx_messageInfo_IdRequest.Marshal(b, m, deterministic)
|
||||||
|
}
|
||||||
|
func (m *IdRequest) XXX_Merge(src proto.Message) {
|
||||||
|
xxx_messageInfo_IdRequest.Merge(m, src)
|
||||||
|
}
|
||||||
|
func (m *IdRequest) XXX_Size() int {
|
||||||
|
return xxx_messageInfo_IdRequest.Size(m)
|
||||||
|
}
|
||||||
|
func (m *IdRequest) XXX_DiscardUnknown() {
|
||||||
|
xxx_messageInfo_IdRequest.DiscardUnknown(m)
|
||||||
|
}
|
||||||
|
|
||||||
|
var xxx_messageInfo_IdRequest proto.InternalMessageInfo
|
||||||
|
|
||||||
|
func (m *IdRequest) GetPeerId() int64 {
|
||||||
|
if m != nil {
|
||||||
|
return m.PeerId
|
||||||
|
}
|
||||||
|
return 0
|
||||||
|
}
|
||||||
|
|
||||||
|
func (m *IdRequest) GetRole() PeerRole {
|
||||||
|
if m != nil {
|
||||||
|
return m.Role
|
||||||
|
}
|
||||||
|
return PeerRole_Master
|
||||||
|
}
|
||||||
|
|
||||||
|
func (m *IdRequest) GetCount() uint32 {
|
||||||
|
if m != nil {
|
||||||
|
return m.Count
|
||||||
|
}
|
||||||
|
return 0
|
||||||
|
}
|
||||||
|
|
||||||
|
type IdResponse struct {
|
||||||
|
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
|
||||||
|
Id int64 `protobuf:"varint,2,opt,name=id,proto3" json:"id,omitempty"`
|
||||||
|
Count uint32 `protobuf:"varint,3,opt,name=count,proto3" json:"count,omitempty"`
|
||||||
|
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||||
|
XXX_unrecognized []byte `json:"-"`
|
||||||
|
XXX_sizecache int32 `json:"-"`
|
||||||
|
}
|
||||||
|
|
||||||
|
func (m *IdResponse) Reset() { *m = IdResponse{} }
|
||||||
|
func (m *IdResponse) String() string { return proto.CompactTextString(m) }
|
||||||
|
func (*IdResponse) ProtoMessage() {}
|
||||||
|
func (*IdResponse) Descriptor() ([]byte, []int) {
|
||||||
|
return fileDescriptor_7eb37f6b80b23116, []int{1}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (m *IdResponse) XXX_Unmarshal(b []byte) error {
|
||||||
|
return xxx_messageInfo_IdResponse.Unmarshal(m, b)
|
||||||
|
}
|
||||||
|
func (m *IdResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||||
|
return xxx_messageInfo_IdResponse.Marshal(b, m, deterministic)
|
||||||
|
}
|
||||||
|
func (m *IdResponse) XXX_Merge(src proto.Message) {
|
||||||
|
xxx_messageInfo_IdResponse.Merge(m, src)
|
||||||
|
}
|
||||||
|
func (m *IdResponse) XXX_Size() int {
|
||||||
|
return xxx_messageInfo_IdResponse.Size(m)
|
||||||
|
}
|
||||||
|
func (m *IdResponse) XXX_DiscardUnknown() {
|
||||||
|
xxx_messageInfo_IdResponse.DiscardUnknown(m)
|
||||||
|
}
|
||||||
|
|
||||||
|
var xxx_messageInfo_IdResponse proto.InternalMessageInfo
|
||||||
|
|
||||||
|
func (m *IdResponse) GetStatus() *commonpb.Status {
|
||||||
|
if m != nil {
|
||||||
|
return m.Status
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (m *IdResponse) GetId() int64 {
|
||||||
|
if m != nil {
|
||||||
|
return m.Id
|
||||||
|
}
|
||||||
|
return 0
|
||||||
|
}
|
||||||
|
|
||||||
|
func (m *IdResponse) GetCount() uint32 {
|
||||||
|
if m != nil {
|
||||||
|
return m.Count
|
||||||
|
}
|
||||||
|
return 0
|
||||||
|
}
|
||||||
|
|
||||||
type TsoRequest struct {
|
type TsoRequest struct {
|
||||||
PeerId int64 `protobuf:"varint,1,opt,name=peer_id,json=peerId,proto3" json:"peer_id,omitempty"`
|
PeerId int64 `protobuf:"varint,1,opt,name=peer_id,json=peerId,proto3" json:"peer_id,omitempty"`
|
||||||
Role PeerRole `protobuf:"varint,2,opt,name=role,proto3,enum=milvus.proto.internal.PeerRole" json:"role,omitempty"`
|
Role PeerRole `protobuf:"varint,2,opt,name=role,proto3,enum=milvus.proto.internal.PeerRole" json:"role,omitempty"`
|
||||||
@ -135,7 +245,7 @@ func (m *TsoRequest) Reset() { *m = TsoRequest{} }
|
|||||||
func (m *TsoRequest) String() string { return proto.CompactTextString(m) }
|
func (m *TsoRequest) String() string { return proto.CompactTextString(m) }
|
||||||
func (*TsoRequest) ProtoMessage() {}
|
func (*TsoRequest) ProtoMessage() {}
|
||||||
func (*TsoRequest) Descriptor() ([]byte, []int) {
|
func (*TsoRequest) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_7eb37f6b80b23116, []int{0}
|
return fileDescriptor_7eb37f6b80b23116, []int{2}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *TsoRequest) XXX_Unmarshal(b []byte) error {
|
func (m *TsoRequest) XXX_Unmarshal(b []byte) error {
|
||||||
@ -190,7 +300,7 @@ func (m *TsoResponse) Reset() { *m = TsoResponse{} }
|
|||||||
func (m *TsoResponse) String() string { return proto.CompactTextString(m) }
|
func (m *TsoResponse) String() string { return proto.CompactTextString(m) }
|
||||||
func (*TsoResponse) ProtoMessage() {}
|
func (*TsoResponse) ProtoMessage() {}
|
||||||
func (*TsoResponse) Descriptor() ([]byte, []int) {
|
func (*TsoResponse) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_7eb37f6b80b23116, []int{1}
|
return fileDescriptor_7eb37f6b80b23116, []int{3}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *TsoResponse) XXX_Unmarshal(b []byte) error {
|
func (m *TsoResponse) XXX_Unmarshal(b []byte) error {
|
||||||
@ -247,7 +357,7 @@ func (m *CreateCollectionRequest) Reset() { *m = CreateCollectionRequest
|
|||||||
func (m *CreateCollectionRequest) String() string { return proto.CompactTextString(m) }
|
func (m *CreateCollectionRequest) String() string { return proto.CompactTextString(m) }
|
||||||
func (*CreateCollectionRequest) ProtoMessage() {}
|
func (*CreateCollectionRequest) ProtoMessage() {}
|
||||||
func (*CreateCollectionRequest) Descriptor() ([]byte, []int) {
|
func (*CreateCollectionRequest) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_7eb37f6b80b23116, []int{2}
|
return fileDescriptor_7eb37f6b80b23116, []int{4}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *CreateCollectionRequest) XXX_Unmarshal(b []byte) error {
|
func (m *CreateCollectionRequest) XXX_Unmarshal(b []byte) error {
|
||||||
@ -318,7 +428,7 @@ func (m *DropCollectionRequest) Reset() { *m = DropCollectionRequest{} }
|
|||||||
func (m *DropCollectionRequest) String() string { return proto.CompactTextString(m) }
|
func (m *DropCollectionRequest) String() string { return proto.CompactTextString(m) }
|
||||||
func (*DropCollectionRequest) ProtoMessage() {}
|
func (*DropCollectionRequest) ProtoMessage() {}
|
||||||
func (*DropCollectionRequest) Descriptor() ([]byte, []int) {
|
func (*DropCollectionRequest) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_7eb37f6b80b23116, []int{3}
|
return fileDescriptor_7eb37f6b80b23116, []int{5}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *DropCollectionRequest) XXX_Unmarshal(b []byte) error {
|
func (m *DropCollectionRequest) XXX_Unmarshal(b []byte) error {
|
||||||
@ -389,7 +499,7 @@ func (m *HasCollectionRequest) Reset() { *m = HasCollectionRequest{} }
|
|||||||
func (m *HasCollectionRequest) String() string { return proto.CompactTextString(m) }
|
func (m *HasCollectionRequest) String() string { return proto.CompactTextString(m) }
|
||||||
func (*HasCollectionRequest) ProtoMessage() {}
|
func (*HasCollectionRequest) ProtoMessage() {}
|
||||||
func (*HasCollectionRequest) Descriptor() ([]byte, []int) {
|
func (*HasCollectionRequest) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_7eb37f6b80b23116, []int{4}
|
return fileDescriptor_7eb37f6b80b23116, []int{6}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *HasCollectionRequest) XXX_Unmarshal(b []byte) error {
|
func (m *HasCollectionRequest) XXX_Unmarshal(b []byte) error {
|
||||||
@ -460,7 +570,7 @@ func (m *DescribeCollectionRequest) Reset() { *m = DescribeCollectionReq
|
|||||||
func (m *DescribeCollectionRequest) String() string { return proto.CompactTextString(m) }
|
func (m *DescribeCollectionRequest) String() string { return proto.CompactTextString(m) }
|
||||||
func (*DescribeCollectionRequest) ProtoMessage() {}
|
func (*DescribeCollectionRequest) ProtoMessage() {}
|
||||||
func (*DescribeCollectionRequest) Descriptor() ([]byte, []int) {
|
func (*DescribeCollectionRequest) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_7eb37f6b80b23116, []int{5}
|
return fileDescriptor_7eb37f6b80b23116, []int{7}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *DescribeCollectionRequest) XXX_Unmarshal(b []byte) error {
|
func (m *DescribeCollectionRequest) XXX_Unmarshal(b []byte) error {
|
||||||
@ -530,7 +640,7 @@ func (m *ShowCollectionRequest) Reset() { *m = ShowCollectionRequest{} }
|
|||||||
func (m *ShowCollectionRequest) String() string { return proto.CompactTextString(m) }
|
func (m *ShowCollectionRequest) String() string { return proto.CompactTextString(m) }
|
||||||
func (*ShowCollectionRequest) ProtoMessage() {}
|
func (*ShowCollectionRequest) ProtoMessage() {}
|
||||||
func (*ShowCollectionRequest) Descriptor() ([]byte, []int) {
|
func (*ShowCollectionRequest) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_7eb37f6b80b23116, []int{6}
|
return fileDescriptor_7eb37f6b80b23116, []int{8}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *ShowCollectionRequest) XXX_Unmarshal(b []byte) error {
|
func (m *ShowCollectionRequest) XXX_Unmarshal(b []byte) error {
|
||||||
@ -594,7 +704,7 @@ func (m *CreatePartitionRequest) Reset() { *m = CreatePartitionRequest{}
|
|||||||
func (m *CreatePartitionRequest) String() string { return proto.CompactTextString(m) }
|
func (m *CreatePartitionRequest) String() string { return proto.CompactTextString(m) }
|
||||||
func (*CreatePartitionRequest) ProtoMessage() {}
|
func (*CreatePartitionRequest) ProtoMessage() {}
|
||||||
func (*CreatePartitionRequest) Descriptor() ([]byte, []int) {
|
func (*CreatePartitionRequest) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_7eb37f6b80b23116, []int{7}
|
return fileDescriptor_7eb37f6b80b23116, []int{9}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *CreatePartitionRequest) XXX_Unmarshal(b []byte) error {
|
func (m *CreatePartitionRequest) XXX_Unmarshal(b []byte) error {
|
||||||
@ -665,7 +775,7 @@ func (m *DropPartitionRequest) Reset() { *m = DropPartitionRequest{} }
|
|||||||
func (m *DropPartitionRequest) String() string { return proto.CompactTextString(m) }
|
func (m *DropPartitionRequest) String() string { return proto.CompactTextString(m) }
|
||||||
func (*DropPartitionRequest) ProtoMessage() {}
|
func (*DropPartitionRequest) ProtoMessage() {}
|
||||||
func (*DropPartitionRequest) Descriptor() ([]byte, []int) {
|
func (*DropPartitionRequest) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_7eb37f6b80b23116, []int{8}
|
return fileDescriptor_7eb37f6b80b23116, []int{10}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *DropPartitionRequest) XXX_Unmarshal(b []byte) error {
|
func (m *DropPartitionRequest) XXX_Unmarshal(b []byte) error {
|
||||||
@ -736,7 +846,7 @@ func (m *HasPartitionRequest) Reset() { *m = HasPartitionRequest{} }
|
|||||||
func (m *HasPartitionRequest) String() string { return proto.CompactTextString(m) }
|
func (m *HasPartitionRequest) String() string { return proto.CompactTextString(m) }
|
||||||
func (*HasPartitionRequest) ProtoMessage() {}
|
func (*HasPartitionRequest) ProtoMessage() {}
|
||||||
func (*HasPartitionRequest) Descriptor() ([]byte, []int) {
|
func (*HasPartitionRequest) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_7eb37f6b80b23116, []int{9}
|
return fileDescriptor_7eb37f6b80b23116, []int{11}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *HasPartitionRequest) XXX_Unmarshal(b []byte) error {
|
func (m *HasPartitionRequest) XXX_Unmarshal(b []byte) error {
|
||||||
@ -807,7 +917,7 @@ func (m *DescribePartitionRequest) Reset() { *m = DescribePartitionReque
|
|||||||
func (m *DescribePartitionRequest) String() string { return proto.CompactTextString(m) }
|
func (m *DescribePartitionRequest) String() string { return proto.CompactTextString(m) }
|
||||||
func (*DescribePartitionRequest) ProtoMessage() {}
|
func (*DescribePartitionRequest) ProtoMessage() {}
|
||||||
func (*DescribePartitionRequest) Descriptor() ([]byte, []int) {
|
func (*DescribePartitionRequest) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_7eb37f6b80b23116, []int{10}
|
return fileDescriptor_7eb37f6b80b23116, []int{12}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *DescribePartitionRequest) XXX_Unmarshal(b []byte) error {
|
func (m *DescribePartitionRequest) XXX_Unmarshal(b []byte) error {
|
||||||
@ -878,7 +988,7 @@ func (m *ShowPartitionRequest) Reset() { *m = ShowPartitionRequest{} }
|
|||||||
func (m *ShowPartitionRequest) String() string { return proto.CompactTextString(m) }
|
func (m *ShowPartitionRequest) String() string { return proto.CompactTextString(m) }
|
||||||
func (*ShowPartitionRequest) ProtoMessage() {}
|
func (*ShowPartitionRequest) ProtoMessage() {}
|
||||||
func (*ShowPartitionRequest) Descriptor() ([]byte, []int) {
|
func (*ShowPartitionRequest) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_7eb37f6b80b23116, []int{11}
|
return fileDescriptor_7eb37f6b80b23116, []int{13}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *ShowPartitionRequest) XXX_Unmarshal(b []byte) error {
|
func (m *ShowPartitionRequest) XXX_Unmarshal(b []byte) error {
|
||||||
@ -954,7 +1064,7 @@ func (m *InsertRequest) Reset() { *m = InsertRequest{} }
|
|||||||
func (m *InsertRequest) String() string { return proto.CompactTextString(m) }
|
func (m *InsertRequest) String() string { return proto.CompactTextString(m) }
|
||||||
func (*InsertRequest) ProtoMessage() {}
|
func (*InsertRequest) ProtoMessage() {}
|
||||||
func (*InsertRequest) Descriptor() ([]byte, []int) {
|
func (*InsertRequest) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_7eb37f6b80b23116, []int{12}
|
return fileDescriptor_7eb37f6b80b23116, []int{14}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *InsertRequest) XXX_Unmarshal(b []byte) error {
|
func (m *InsertRequest) XXX_Unmarshal(b []byte) error {
|
||||||
@ -1062,7 +1172,7 @@ func (m *DeleteRequest) Reset() { *m = DeleteRequest{} }
|
|||||||
func (m *DeleteRequest) String() string { return proto.CompactTextString(m) }
|
func (m *DeleteRequest) String() string { return proto.CompactTextString(m) }
|
||||||
func (*DeleteRequest) ProtoMessage() {}
|
func (*DeleteRequest) ProtoMessage() {}
|
||||||
func (*DeleteRequest) Descriptor() ([]byte, []int) {
|
func (*DeleteRequest) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_7eb37f6b80b23116, []int{13}
|
return fileDescriptor_7eb37f6b80b23116, []int{15}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *DeleteRequest) XXX_Unmarshal(b []byte) error {
|
func (m *DeleteRequest) XXX_Unmarshal(b []byte) error {
|
||||||
@ -1148,7 +1258,7 @@ func (m *SearchRequest) Reset() { *m = SearchRequest{} }
|
|||||||
func (m *SearchRequest) String() string { return proto.CompactTextString(m) }
|
func (m *SearchRequest) String() string { return proto.CompactTextString(m) }
|
||||||
func (*SearchRequest) ProtoMessage() {}
|
func (*SearchRequest) ProtoMessage() {}
|
||||||
func (*SearchRequest) Descriptor() ([]byte, []int) {
|
func (*SearchRequest) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_7eb37f6b80b23116, []int{14}
|
return fileDescriptor_7eb37f6b80b23116, []int{16}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *SearchRequest) XXX_Unmarshal(b []byte) error {
|
func (m *SearchRequest) XXX_Unmarshal(b []byte) error {
|
||||||
@ -1228,7 +1338,7 @@ func (m *SearchResult) Reset() { *m = SearchResult{} }
|
|||||||
func (m *SearchResult) String() string { return proto.CompactTextString(m) }
|
func (m *SearchResult) String() string { return proto.CompactTextString(m) }
|
||||||
func (*SearchResult) ProtoMessage() {}
|
func (*SearchResult) ProtoMessage() {}
|
||||||
func (*SearchResult) Descriptor() ([]byte, []int) {
|
func (*SearchResult) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_7eb37f6b80b23116, []int{15}
|
return fileDescriptor_7eb37f6b80b23116, []int{17}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *SearchResult) XXX_Unmarshal(b []byte) error {
|
func (m *SearchResult) XXX_Unmarshal(b []byte) error {
|
||||||
@ -1310,7 +1420,7 @@ func (m *TimeSyncMsg) Reset() { *m = TimeSyncMsg{} }
|
|||||||
func (m *TimeSyncMsg) String() string { return proto.CompactTextString(m) }
|
func (m *TimeSyncMsg) String() string { return proto.CompactTextString(m) }
|
||||||
func (*TimeSyncMsg) ProtoMessage() {}
|
func (*TimeSyncMsg) ProtoMessage() {}
|
||||||
func (*TimeSyncMsg) Descriptor() ([]byte, []int) {
|
func (*TimeSyncMsg) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_7eb37f6b80b23116, []int{16}
|
return fileDescriptor_7eb37f6b80b23116, []int{18}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *TimeSyncMsg) XXX_Unmarshal(b []byte) error {
|
func (m *TimeSyncMsg) XXX_Unmarshal(b []byte) error {
|
||||||
@ -1360,7 +1470,7 @@ func (m *Key2Seg) Reset() { *m = Key2Seg{} }
|
|||||||
func (m *Key2Seg) String() string { return proto.CompactTextString(m) }
|
func (m *Key2Seg) String() string { return proto.CompactTextString(m) }
|
||||||
func (*Key2Seg) ProtoMessage() {}
|
func (*Key2Seg) ProtoMessage() {}
|
||||||
func (*Key2Seg) Descriptor() ([]byte, []int) {
|
func (*Key2Seg) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_7eb37f6b80b23116, []int{17}
|
return fileDescriptor_7eb37f6b80b23116, []int{19}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *Key2Seg) XXX_Unmarshal(b []byte) error {
|
func (m *Key2Seg) XXX_Unmarshal(b []byte) error {
|
||||||
@ -1428,7 +1538,7 @@ func (m *Key2SegMsg) Reset() { *m = Key2SegMsg{} }
|
|||||||
func (m *Key2SegMsg) String() string { return proto.CompactTextString(m) }
|
func (m *Key2SegMsg) String() string { return proto.CompactTextString(m) }
|
||||||
func (*Key2SegMsg) ProtoMessage() {}
|
func (*Key2SegMsg) ProtoMessage() {}
|
||||||
func (*Key2SegMsg) Descriptor() ([]byte, []int) {
|
func (*Key2SegMsg) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_7eb37f6b80b23116, []int{18}
|
return fileDescriptor_7eb37f6b80b23116, []int{20}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *Key2SegMsg) XXX_Unmarshal(b []byte) error {
|
func (m *Key2SegMsg) XXX_Unmarshal(b []byte) error {
|
||||||
@ -1476,7 +1586,7 @@ func (m *SegmentStatistics) Reset() { *m = SegmentStatistics{} }
|
|||||||
func (m *SegmentStatistics) String() string { return proto.CompactTextString(m) }
|
func (m *SegmentStatistics) String() string { return proto.CompactTextString(m) }
|
||||||
func (*SegmentStatistics) ProtoMessage() {}
|
func (*SegmentStatistics) ProtoMessage() {}
|
||||||
func (*SegmentStatistics) Descriptor() ([]byte, []int) {
|
func (*SegmentStatistics) Descriptor() ([]byte, []int) {
|
||||||
return fileDescriptor_7eb37f6b80b23116, []int{19}
|
return fileDescriptor_7eb37f6b80b23116, []int{21}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *SegmentStatistics) XXX_Unmarshal(b []byte) error {
|
func (m *SegmentStatistics) XXX_Unmarshal(b []byte) error {
|
||||||
@ -1521,6 +1631,8 @@ func (m *SegmentStatistics) GetNumRows() int64 {
|
|||||||
func init() {
|
func init() {
|
||||||
proto.RegisterEnum("milvus.proto.internal.ReqType", ReqType_name, ReqType_value)
|
proto.RegisterEnum("milvus.proto.internal.ReqType", ReqType_name, ReqType_value)
|
||||||
proto.RegisterEnum("milvus.proto.internal.PeerRole", PeerRole_name, PeerRole_value)
|
proto.RegisterEnum("milvus.proto.internal.PeerRole", PeerRole_name, PeerRole_value)
|
||||||
|
proto.RegisterType((*IdRequest)(nil), "milvus.proto.internal.IdRequest")
|
||||||
|
proto.RegisterType((*IdResponse)(nil), "milvus.proto.internal.IdResponse")
|
||||||
proto.RegisterType((*TsoRequest)(nil), "milvus.proto.internal.TsoRequest")
|
proto.RegisterType((*TsoRequest)(nil), "milvus.proto.internal.TsoRequest")
|
||||||
proto.RegisterType((*TsoResponse)(nil), "milvus.proto.internal.TsoResponse")
|
proto.RegisterType((*TsoResponse)(nil), "milvus.proto.internal.TsoResponse")
|
||||||
proto.RegisterType((*CreateCollectionRequest)(nil), "milvus.proto.internal.CreateCollectionRequest")
|
proto.RegisterType((*CreateCollectionRequest)(nil), "milvus.proto.internal.CreateCollectionRequest")
|
||||||
@ -1546,77 +1658,79 @@ func init() {
|
|||||||
func init() { proto.RegisterFile("internal_msg.proto", fileDescriptor_7eb37f6b80b23116) }
|
func init() { proto.RegisterFile("internal_msg.proto", fileDescriptor_7eb37f6b80b23116) }
|
||||||
|
|
||||||
var fileDescriptor_7eb37f6b80b23116 = []byte{
|
var fileDescriptor_7eb37f6b80b23116 = []byte{
|
||||||
// 1147 bytes of a gzipped FileDescriptorProto
|
// 1171 bytes of a gzipped FileDescriptorProto
|
||||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x58, 0x49, 0x6f, 0x23, 0x45,
|
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x58, 0x4b, 0x6f, 0x1b, 0x55,
|
||||||
0x14, 0x9e, 0xf6, 0xee, 0xe7, 0xd8, 0xe9, 0x54, 0x62, 0xe2, 0x19, 0x60, 0xc6, 0xf4, 0x20, 0x11,
|
0x14, 0xee, 0xf8, 0x19, 0x1f, 0xc7, 0xce, 0xe4, 0x26, 0x21, 0x6e, 0x81, 0xd6, 0x4c, 0x91, 0x88,
|
||||||
0x8d, 0x84, 0x23, 0x3c, 0x1c, 0x98, 0xeb, 0xc4, 0x87, 0x98, 0x51, 0xa2, 0xa8, 0x1d, 0x81, 0x84,
|
0x2a, 0xe1, 0x08, 0x97, 0x05, 0xdd, 0xb6, 0x5e, 0xd4, 0x54, 0xad, 0xaa, 0x71, 0x04, 0x12, 0x12,
|
||||||
0x84, 0x5a, 0xed, 0xee, 0x87, 0x5d, 0xea, 0x35, 0x55, 0xe5, 0x04, 0xe7, 0x0f, 0x70, 0x05, 0x71,
|
0x1a, 0x8d, 0x67, 0x0e, 0xf6, 0xd5, 0x3c, 0xee, 0xe4, 0xde, 0xeb, 0x04, 0xe7, 0x0f, 0xb0, 0x05,
|
||||||
0xe4, 0xc6, 0x99, 0x03, 0x3f, 0x83, 0xed, 0xce, 0x9f, 0x00, 0xc1, 0x48, 0xa0, 0xb9, 0xa2, 0xaa,
|
0xb1, 0x64, 0xc7, 0x9a, 0x05, 0x3f, 0x83, 0xd7, 0x9e, 0x3f, 0x01, 0x82, 0x4a, 0xa0, 0x6e, 0xd1,
|
||||||
0x6e, 0x2f, 0xed, 0x2c, 0x2c, 0x62, 0xa4, 0x48, 0xb9, 0xf5, 0x7b, 0xb5, 0xbd, 0xef, 0x7b, 0x8b,
|
0xbd, 0x33, 0x7e, 0x8c, 0xf3, 0xe0, 0x59, 0x14, 0x29, 0xbb, 0x39, 0xe7, 0x3e, 0xce, 0xf9, 0xbe,
|
||||||
0xdf, 0x33, 0x10, 0x1a, 0x0a, 0x64, 0xa1, 0xed, 0x5b, 0x01, 0x1f, 0x75, 0x62, 0x16, 0x89, 0x88,
|
0xf3, 0xf0, 0x3d, 0x06, 0x42, 0x63, 0x89, 0x3c, 0x76, 0x43, 0x27, 0x12, 0xa3, 0x4e, 0xc2, 0x99,
|
||||||
0x34, 0x03, 0xea, 0x9f, 0x4e, 0x78, 0x22, 0x75, 0x66, 0x1b, 0xee, 0xad, 0x39, 0x51, 0x10, 0x44,
|
0x64, 0x64, 0x27, 0xa2, 0xe1, 0xd1, 0x44, 0xa4, 0x52, 0x67, 0xb6, 0xe1, 0xc6, 0xba, 0xc7, 0xa2,
|
||||||
0x61, 0xa2, 0xbe, 0xb7, 0xc1, 0x91, 0x9d, 0x52, 0x07, 0x17, 0xe7, 0x8c, 0x18, 0xe0, 0x98, 0x47,
|
0x88, 0xc5, 0xa9, 0xfa, 0xc6, 0xa6, 0x40, 0x7e, 0x44, 0x3d, 0x5c, 0x9c, 0xb3, 0x18, 0xd4, 0xfa,
|
||||||
0x26, 0x9e, 0x4c, 0x90, 0x0b, 0xb2, 0x0d, 0xe5, 0x18, 0x91, 0x59, 0xd4, 0x6d, 0x69, 0x6d, 0x6d,
|
0xbe, 0x8d, 0x87, 0x13, 0x14, 0x92, 0xec, 0x42, 0x35, 0x41, 0xe4, 0x0e, 0xf5, 0x5b, 0x46, 0xdb,
|
||||||
0x27, 0x6f, 0x96, 0xa4, 0xd8, 0x77, 0xc9, 0x63, 0x28, 0xb0, 0xc8, 0xc7, 0x56, 0xae, 0xad, 0xed,
|
0xd8, 0x2b, 0xda, 0x15, 0x25, 0xf6, 0x7d, 0x72, 0x17, 0x4a, 0x9c, 0x85, 0xd8, 0x2a, 0xb4, 0x8d,
|
||||||
0x34, 0xba, 0x0f, 0x3a, 0x97, 0xbe, 0xd6, 0x39, 0x42, 0x64, 0x66, 0xe4, 0xa3, 0xa9, 0x36, 0x93,
|
0xbd, 0x66, 0xf7, 0x56, 0xe7, 0x4c, 0x63, 0x9d, 0xa7, 0x88, 0xdc, 0x66, 0x21, 0xda, 0x7a, 0x33,
|
||||||
0x2d, 0x28, 0x3a, 0xd1, 0x24, 0x14, 0xad, 0x7c, 0x5b, 0xdb, 0xa9, 0x9b, 0x89, 0x60, 0x9c, 0x42,
|
0xd9, 0x86, 0xb2, 0xc7, 0x26, 0xb1, 0x6c, 0x15, 0xdb, 0xc6, 0x5e, 0xc3, 0x4e, 0x05, 0x6b, 0x04,
|
||||||
0x4d, 0xbd, 0xc8, 0xe3, 0x28, 0xe4, 0x48, 0x1e, 0x43, 0x89, 0x0b, 0x5b, 0x4c, 0xb8, 0x7a, 0xb1,
|
0xa0, 0x0c, 0x8a, 0x84, 0xc5, 0x02, 0xc9, 0x5d, 0xa8, 0x08, 0xe9, 0xca, 0x89, 0xd0, 0x06, 0xeb,
|
||||||
0xd6, 0x7d, 0x35, 0x7b, 0x77, 0x6a, 0xff, 0x40, 0x6d, 0x31, 0xd3, 0xad, 0xe4, 0x35, 0xa8, 0x0a,
|
0xdd, 0x97, 0xf3, 0x57, 0x67, 0xde, 0x0f, 0xf4, 0x16, 0x3b, 0xdb, 0x4a, 0x9a, 0x50, 0xa0, 0xbe,
|
||||||
0x1a, 0x20, 0x17, 0x76, 0x10, 0x2b, 0x9b, 0x0a, 0xe6, 0x42, 0x71, 0xc5, 0xbb, 0x3f, 0x6b, 0xb0,
|
0xf6, 0xa5, 0x68, 0x17, 0xa8, 0x7f, 0x8e, 0xa1, 0x04, 0xe0, 0x40, 0xb0, 0xff, 0x13, 0xda, 0x11,
|
||||||
0xbd, 0xc7, 0xd0, 0x16, 0xb8, 0x17, 0xf9, 0x3e, 0x3a, 0x82, 0x46, 0xe1, 0x0c, 0xf7, 0x13, 0xa8,
|
0xd4, 0xb5, 0xc5, 0x7f, 0x83, 0xed, 0x15, 0xa8, 0x49, 0x1a, 0xa1, 0x90, 0x6e, 0x94, 0x68, 0x9f,
|
||||||
0x30, 0x3c, 0xb1, 0xc4, 0x34, 0x46, 0x65, 0x46, 0xa3, 0x7b, 0xff, 0x0a, 0x88, 0x26, 0x9e, 0x1c,
|
0x4a, 0xf6, 0x42, 0x71, 0x8e, 0xdd, 0x1f, 0x0d, 0xd8, 0x7d, 0xc0, 0xd1, 0x95, 0xf8, 0x80, 0x85,
|
||||||
0x4f, 0x63, 0x34, 0xcb, 0x2c, 0xf9, 0x20, 0x4d, 0x28, 0xc9, 0xa3, 0xd4, 0x55, 0x76, 0xe4, 0xcd,
|
0x21, 0x7a, 0x92, 0xb2, 0x78, 0x86, 0xfb, 0x1e, 0xac, 0x71, 0x3c, 0x74, 0xe4, 0x34, 0x41, 0xed,
|
||||||
0x22, 0xc3, 0x93, 0xbe, 0x9b, 0xb5, 0x30, 0xbf, 0x6a, 0xe1, 0x5d, 0xa8, 0xc4, 0x2c, 0xfa, 0x74,
|
0x46, 0xb3, 0x7b, 0xf3, 0x1c, 0x88, 0x36, 0x1e, 0x1e, 0x4c, 0x13, 0xb4, 0xab, 0x3c, 0xfd, 0x20,
|
||||||
0x2a, 0x8f, 0x15, 0xd4, 0xb1, 0xb2, 0x92, 0xfb, 0x2e, 0x79, 0x07, 0x4a, 0xdc, 0x19, 0x63, 0x60,
|
0x3b, 0x50, 0x51, 0x47, 0xe7, 0x54, 0x97, 0x39, 0x1e, 0xf6, 0xfd, 0xbc, 0x87, 0xc5, 0x55, 0x0f,
|
||||||
0xb7, 0x8a, 0x8a, 0x8f, 0xbb, 0x97, 0xf2, 0xf1, 0xd4, 0x8f, 0x86, 0x66, 0xba, 0xd1, 0x78, 0xae,
|
0xaf, 0xc3, 0x5a, 0xc2, 0xd9, 0xc7, 0x53, 0x75, 0xac, 0xa4, 0x8f, 0x55, 0xb5, 0xdc, 0xf7, 0xc9,
|
||||||
0x41, 0xb3, 0xc7, 0xa2, 0xf8, 0x46, 0xe3, 0x3a, 0x80, 0x75, 0x67, 0x6e, 0x9f, 0x15, 0xda, 0x01,
|
0x5b, 0x50, 0x11, 0xde, 0x18, 0x23, 0xb7, 0x55, 0xd6, 0x7c, 0x5c, 0x3f, 0x93, 0x8f, 0xfb, 0x21,
|
||||||
0xa6, 0x00, 0xdf, 0xcc, 0x5a, 0x94, 0x86, 0x68, 0x67, 0x01, 0xe6, 0xd0, 0x0e, 0xd0, 0x6c, 0x38,
|
0x1b, 0xda, 0xd9, 0x46, 0xeb, 0x99, 0x01, 0x3b, 0x3d, 0xce, 0x92, 0x4b, 0x8d, 0xeb, 0x31, 0x6c,
|
||||||
0x19, 0xd9, 0xf8, 0x5d, 0x83, 0xad, 0x7d, 0x9b, 0xdf, 0x26, 0xc8, 0x7f, 0x6a, 0x70, 0xb7, 0x87,
|
0x78, 0x73, 0xff, 0x9c, 0xd8, 0x8d, 0x30, 0x03, 0xf8, 0x7a, 0xde, 0xa3, 0xac, 0xf8, 0x3a, 0x0b,
|
||||||
0xdc, 0x61, 0x74, 0x88, 0xb7, 0x09, 0xf7, 0xd7, 0x1a, 0x34, 0x07, 0xe3, 0xe8, 0xec, 0x26, 0x63,
|
0x30, 0x4f, 0xdc, 0x08, 0xed, 0xa6, 0x97, 0x93, 0xad, 0x5f, 0x0d, 0xd8, 0x7e, 0xe8, 0x8a, 0xab,
|
||||||
0x36, 0x7e, 0xd3, 0xe0, 0x95, 0xa4, 0xba, 0x1c, 0xd9, 0x4c, 0xd0, 0x1b, 0xea, 0x99, 0xf7, 0xa1,
|
0x04, 0xf9, 0x77, 0x03, 0xae, 0xf7, 0x50, 0x78, 0x9c, 0x0e, 0xf1, 0x2a, 0xe1, 0xfe, 0xd2, 0x80,
|
||||||
0x11, 0xcf, 0xcc, 0x5b, 0x76, 0xcc, 0xc3, 0xcb, 0x1d, 0x33, 0x87, 0xa2, 0xfc, 0x52, 0x8f, 0x97,
|
0x9d, 0xc1, 0x98, 0x1d, 0x5f, 0x66, 0xcc, 0xd6, 0x2f, 0x06, 0xbc, 0x94, 0x76, 0x97, 0xa7, 0x2e,
|
||||||
0x45, 0xe3, 0x57, 0x0d, 0xb6, 0x64, 0xd5, 0xb9, 0x2d, 0x78, 0x7f, 0xd1, 0x60, 0x73, 0xdf, 0xe6,
|
0x97, 0xf4, 0x92, 0x46, 0xe6, 0x5d, 0x68, 0x26, 0x33, 0xf7, 0x96, 0x03, 0x73, 0xfb, 0xec, 0xc0,
|
||||||
0xb7, 0x05, 0xee, 0x73, 0x0d, 0x5a, 0xb3, 0x6a, 0x73, 0x5b, 0x30, 0xcb, 0x1f, 0x15, 0x59, 0x69,
|
0xcc, 0xa1, 0xe8, 0xb8, 0x34, 0x92, 0x65, 0xd1, 0xfa, 0xd9, 0x80, 0x6d, 0xd5, 0x75, 0xae, 0x0a,
|
||||||
0x6e, 0x32, 0xde, 0xff, 0xb9, 0xb8, 0xbe, 0xc8, 0x41, 0xbd, 0x1f, 0x72, 0x64, 0xe2, 0xe5, 0x61,
|
0xde, 0x9f, 0x0c, 0xd8, 0x7a, 0xe8, 0x8a, 0xab, 0x02, 0xf7, 0x99, 0x01, 0xad, 0x59, 0xb7, 0xb9,
|
||||||
0x7d, 0xeb, 0xa2, 0xc9, 0x12, 0x71, 0x75, 0xd5, 0x18, 0xf2, 0x10, 0x16, 0x0e, 0xb1, 0x84, 0x3d,
|
0x2a, 0x98, 0xd5, 0x8f, 0x8a, 0xea, 0x34, 0x97, 0x19, 0xef, 0x7f, 0xdc, 0x5c, 0x9f, 0x17, 0xa0,
|
||||||
0x52, 0xd8, 0xab, 0xe6, 0xda, 0x5c, 0x79, 0x6c, 0x8f, 0xc8, 0xeb, 0x00, 0x1c, 0x47, 0x01, 0x86,
|
0xd1, 0x8f, 0x05, 0x72, 0xf9, 0xe2, 0xb0, 0xbe, 0x71, 0xda, 0x65, 0x85, 0xb8, 0xb6, 0xea, 0x0c,
|
||||||
0x42, 0x3e, 0x54, 0x54, 0x0f, 0x55, 0x53, 0x4d, 0xdf, 0x95, 0xcb, 0xce, 0xd8, 0x0e, 0x43, 0xf4,
|
0xb9, 0x0d, 0x8b, 0x80, 0x38, 0xd2, 0x1d, 0x69, 0xec, 0x35, 0x7b, 0x7d, 0xae, 0x3c, 0x70, 0x47,
|
||||||
0xe5, 0x72, 0x29, 0x59, 0x4e, 0x35, 0x7d, 0x37, 0xc3, 0x6c, 0x39, 0xcb, 0xec, 0x7d, 0x80, 0xb9,
|
0xe4, 0x55, 0x00, 0x81, 0xa3, 0x08, 0x63, 0xa9, 0x0c, 0x95, 0xb5, 0xa1, 0x5a, 0xa6, 0xe9, 0xfb,
|
||||||
0x07, 0x78, 0xab, 0xd2, 0xce, 0xef, 0x14, 0xcc, 0x25, 0x8d, 0x6c, 0x8e, 0x59, 0x74, 0x66, 0x51,
|
0x6a, 0xd9, 0x1b, 0xbb, 0x71, 0x8c, 0xa1, 0x5a, 0xae, 0xa4, 0xcb, 0x99, 0xa6, 0xef, 0xe7, 0x98,
|
||||||
0x97, 0xb7, 0xaa, 0xed, 0xbc, 0x6c, 0x8e, 0x59, 0x74, 0xd6, 0x77, 0x39, 0x79, 0x17, 0x2a, 0x72,
|
0xad, 0xe6, 0x99, 0xbd, 0x09, 0x30, 0x8f, 0x80, 0x68, 0xad, 0xb5, 0x8b, 0x7b, 0x25, 0x7b, 0x49,
|
||||||
0xc1, 0xb5, 0x85, 0xdd, 0x82, 0x76, 0xfe, 0xfa, 0xa6, 0x4d, 0xde, 0xd1, 0xb3, 0x85, 0x6d, 0x7c,
|
0xa3, 0x1e, 0xc7, 0x9c, 0x1d, 0x3b, 0xd4, 0x17, 0xad, 0x5a, 0xbb, 0xa8, 0x1e, 0xc7, 0x9c, 0x1d,
|
||||||
0x96, 0x83, 0x7a, 0x0f, 0x7d, 0x14, 0x78, 0x03, 0x98, 0xcf, 0xb2, 0x56, 0xb8, 0x8e, 0xb5, 0xe2,
|
0xf7, 0x7d, 0x41, 0xde, 0x86, 0x35, 0xb5, 0xe0, 0xbb, 0xd2, 0x6d, 0x41, 0xbb, 0x78, 0xf1, 0xa3,
|
||||||
0x75, 0xac, 0x95, 0x2e, 0xb0, 0xf6, 0x06, 0xac, 0xc5, 0x8c, 0x06, 0x36, 0x9b, 0x5a, 0x1e, 0x4e,
|
0x4d, 0xdd, 0xd1, 0x73, 0xa5, 0x6b, 0x7d, 0x52, 0x80, 0x46, 0x0f, 0x43, 0x94, 0x78, 0x09, 0x98,
|
||||||
0x79, 0xab, 0xac, 0xa8, 0xab, 0xa5, 0xba, 0x67, 0x38, 0xe5, 0xc6, 0x0b, 0x0d, 0xea, 0x03, 0xb4,
|
0xcf, 0xb3, 0x56, 0xba, 0x88, 0xb5, 0xf2, 0x45, 0xac, 0x55, 0x4e, 0xb1, 0xf6, 0x1a, 0xac, 0x27,
|
||||||
0x99, 0x33, 0x7e, 0x79, 0x4c, 0x2c, 0x23, 0xc8, 0x67, 0x11, 0x64, 0x52, 0xb1, 0xb0, 0x9a, 0x8a,
|
0x9c, 0x46, 0x2e, 0x9f, 0x3a, 0x01, 0x4e, 0x45, 0xab, 0xaa, 0xa9, 0xab, 0x67, 0xba, 0x47, 0x38,
|
||||||
0x8f, 0x60, 0x83, 0x21, 0x9f, 0xf8, 0xc2, 0x5a, 0x22, 0x28, 0xe1, 0x60, 0x3d, 0x59, 0xd8, 0x9b,
|
0x15, 0xd6, 0x73, 0x03, 0x1a, 0x03, 0x74, 0xb9, 0x37, 0x7e, 0x71, 0x4c, 0x2c, 0x23, 0x28, 0xe6,
|
||||||
0xd3, 0xb4, 0x0b, 0xc5, 0x93, 0x09, 0xb2, 0xa9, 0x0a, 0xbb, 0x6b, 0xa3, 0x20, 0xd9, 0x67, 0x7c,
|
0x11, 0xe4, 0x4a, 0xb1, 0xb4, 0x5a, 0x8a, 0x77, 0x60, 0x93, 0xa3, 0x98, 0x84, 0xd2, 0x59, 0x22,
|
||||||
0x99, 0x83, 0xb5, 0x19, 0x72, 0x79, 0xd5, 0x7f, 0x9b, 0x86, 0xfe, 0x3d, 0x64, 0x03, 0xea, 0xca,
|
0x28, 0xe5, 0x60, 0x23, 0x5d, 0x78, 0x30, 0xa7, 0x69, 0x1f, 0xca, 0x87, 0x13, 0xe4, 0x53, 0x9d,
|
||||||
0x00, 0x2b, 0x8c, 0x5c, 0x5c, 0x78, 0xbc, 0xa6, 0x94, 0x87, 0x91, 0x8b, 0xab, 0xb4, 0x14, 0xff,
|
0x76, 0x17, 0x66, 0x41, 0xba, 0xcf, 0xfa, 0xbc, 0x00, 0xeb, 0x33, 0xe4, 0xea, 0xaa, 0x7f, 0x36,
|
||||||
0x11, 0x2d, 0xa5, 0xcb, 0x69, 0xe9, 0x40, 0x61, 0x4c, 0x45, 0xe2, 0xfa, 0x5a, 0xf7, 0xde, 0xe5,
|
0x0d, 0xfd, 0x7d, 0xc8, 0x16, 0x34, 0xb4, 0x03, 0x4e, 0xcc, 0x7c, 0x5c, 0x44, 0xbc, 0xae, 0x95,
|
||||||
0x75, 0x6a, 0x9f, 0x0a, 0x6e, 0xaa, 0x7d, 0x46, 0x0f, 0x6a, 0xc7, 0x34, 0xc0, 0xc1, 0x34, 0x74,
|
0x4f, 0x98, 0x8f, 0xab, 0xb4, 0x94, 0xff, 0x12, 0x2d, 0x95, 0xb3, 0x69, 0xe9, 0x40, 0x69, 0x4c,
|
||||||
0x0e, 0xf8, 0xe8, 0xea, 0xa1, 0xf4, 0xda, 0x29, 0xd0, 0xf8, 0x4a, 0x83, 0xf2, 0x33, 0x9c, 0x76,
|
0x65, 0x1a, 0xfa, 0x7a, 0xf7, 0xc6, 0xd9, 0x7d, 0xea, 0x21, 0x95, 0xc2, 0xd6, 0xfb, 0xac, 0x1e,
|
||||||
0x07, 0x38, 0x52, 0x0c, 0xa9, 0xd4, 0x4d, 0x6f, 0x28, 0xaa, 0xcc, 0x25, 0x0f, 0xa0, 0xb6, 0x14,
|
0xd4, 0x0f, 0x68, 0x84, 0x83, 0x69, 0xec, 0x3d, 0x16, 0xa3, 0xf3, 0x87, 0xd2, 0x0b, 0xa7, 0x40,
|
||||||
0x9b, 0x29, 0x7b, 0xb0, 0x08, 0xcd, 0xbf, 0xaf, 0xd2, 0x94, 0x5b, 0xa7, 0xb6, 0x9f, 0x12, 0x58,
|
0xeb, 0x0b, 0x03, 0xaa, 0x8f, 0x70, 0xda, 0x1d, 0xe0, 0x48, 0x33, 0xa4, 0x4b, 0x37, 0xbb, 0xa1,
|
||||||
0x31, 0xcb, 0x94, 0x7f, 0x20, 0x45, 0x79, 0xf3, 0xa2, 0x48, 0xf1, 0x56, 0x51, 0x05, 0x3d, 0xcc,
|
0xac, 0x2b, 0x97, 0xdc, 0x82, 0xfa, 0x52, 0x6e, 0x66, 0xec, 0xc1, 0x22, 0x35, 0xff, 0xbc, 0x4b,
|
||||||
0xab, 0x14, 0x37, 0x3e, 0x06, 0x48, 0x8d, 0x93, 0x10, 0x17, 0x1e, 0xd4, 0x96, 0x3d, 0xf8, 0x1e,
|
0x53, 0xe1, 0x1c, 0xb9, 0x61, 0x46, 0xe0, 0x9a, 0x5d, 0xa5, 0xe2, 0x3d, 0x25, 0xaa, 0x9b, 0x17,
|
||||||
0x94, 0x3d, 0x9c, 0x76, 0x39, 0x8e, 0x5a, 0x39, 0xc5, 0xdd, 0x55, 0x59, 0x90, 0x5e, 0x65, 0xce,
|
0x4d, 0x4a, 0xb4, 0xca, 0x3a, 0xe9, 0x61, 0xde, 0xa5, 0x84, 0xf5, 0x21, 0x40, 0xe6, 0x9c, 0x82,
|
||||||
0xb6, 0x1b, 0x21, 0x6c, 0x0c, 0x92, 0xc7, 0x64, 0xac, 0x50, 0x2e, 0xa8, 0xc3, 0x57, 0x2a, 0xa7,
|
0xb8, 0x88, 0xa0, 0xb1, 0x1c, 0xc1, 0x77, 0xa0, 0x1a, 0xe0, 0xb4, 0x2b, 0x70, 0xd4, 0x2a, 0x68,
|
||||||
0xb6, 0x5a, 0x39, 0x1f, 0x40, 0x2d, 0xc0, 0x20, 0x62, 0x53, 0x8b, 0xd3, 0x73, 0x9c, 0xb1, 0x91,
|
0xee, 0xce, 0xab, 0x82, 0xec, 0x2a, 0x7b, 0xb6, 0xdd, 0x8a, 0x61, 0x73, 0x90, 0x1a, 0x53, 0xb9,
|
||||||
0xa8, 0x06, 0xf4, 0x1c, 0x25, 0xde, 0x70, 0x12, 0x58, 0x2c, 0x3a, 0xe3, 0xb3, 0x80, 0x0a, 0x27,
|
0x42, 0x85, 0xa4, 0x9e, 0x58, 0xe9, 0x9c, 0xc6, 0x6a, 0xe7, 0xbc, 0x05, 0xf5, 0x08, 0x23, 0xc6,
|
||||||
0x81, 0x19, 0x9d, 0xf1, 0x47, 0xdf, 0xe4, 0xa0, 0x9c, 0xa6, 0x22, 0xa9, 0x42, 0xd1, 0x3b, 0x8c,
|
0xa7, 0x8e, 0xa0, 0x27, 0x38, 0x63, 0x23, 0x55, 0x0d, 0xe8, 0x09, 0x2a, 0xbc, 0xf1, 0x24, 0x72,
|
||||||
0x42, 0xd4, 0xef, 0x90, 0x26, 0x6c, 0x78, 0xab, 0x33, 0xb7, 0xee, 0x92, 0x4d, 0x58, 0xf7, 0xb2,
|
0x38, 0x3b, 0x16, 0xb3, 0x84, 0x8a, 0x27, 0x91, 0xcd, 0x8e, 0xc5, 0x9d, 0xaf, 0x0a, 0x50, 0xcd,
|
||||||
0x03, 0xab, 0x8e, 0x84, 0x40, 0xc3, 0xcb, 0x4c, 0x74, 0xfa, 0x27, 0x64, 0x1b, 0x36, 0xbd, 0x8b,
|
0x4a, 0x91, 0xd4, 0xa0, 0x1c, 0x3c, 0x61, 0x31, 0x9a, 0xd7, 0xc8, 0x0e, 0x6c, 0x06, 0xab, 0x33,
|
||||||
0x23, 0x8f, 0x3e, 0x22, 0x5b, 0xa0, 0x7b, 0xd9, 0x99, 0x80, 0xeb, 0x63, 0xd2, 0x04, 0xdd, 0x5b,
|
0xb7, 0xe9, 0x93, 0x2d, 0xd8, 0x08, 0xf2, 0x03, 0xab, 0x89, 0x84, 0x40, 0x33, 0xc8, 0x4d, 0x74,
|
||||||
0x69, 0xc2, 0xf5, 0xef, 0x34, 0xb2, 0x09, 0x0d, 0x2f, 0xd3, 0xa9, 0xea, 0xdf, 0x6b, 0x84, 0x40,
|
0xe6, 0x47, 0x64, 0x17, 0xb6, 0x82, 0xd3, 0x23, 0x8f, 0x39, 0x22, 0xdb, 0x60, 0x06, 0xf9, 0x99,
|
||||||
0xdd, 0x5b, 0x6e, 0xe7, 0xf4, 0x1f, 0x34, 0xb2, 0x0d, 0xc4, 0xbb, 0xd0, 0xf3, 0xe8, 0x3f, 0x6a,
|
0x40, 0x98, 0x63, 0xb2, 0x03, 0x66, 0xb0, 0xf2, 0x08, 0x37, 0xbf, 0x31, 0xc8, 0x16, 0x34, 0x83,
|
||||||
0x64, 0x0b, 0xd6, 0xbd, 0x4c, 0x63, 0xc0, 0xf5, 0x9f, 0x34, 0xb2, 0x06, 0x65, 0x2f, 0xf9, 0xed,
|
0xdc, 0x4b, 0xd5, 0xfc, 0xd6, 0x20, 0x04, 0x1a, 0xc1, 0xf2, 0x73, 0xce, 0xfc, 0xce, 0x20, 0xbb,
|
||||||
0xd4, 0x3f, 0xcf, 0x2b, 0x29, 0xa9, 0xe7, 0xfa, 0x17, 0x89, 0x94, 0x64, 0xb6, 0xfe, 0x47, 0x9e,
|
0x40, 0x82, 0x53, 0x6f, 0x1e, 0xf3, 0x7b, 0x83, 0x6c, 0xc3, 0x46, 0x90, 0x7b, 0x18, 0x08, 0xf3,
|
||||||
0x34, 0xa0, 0xea, 0xc9, 0x90, 0x3e, 0xa6, 0x8e, 0xa7, 0x7f, 0x5b, 0x7d, 0xf4, 0x04, 0x2a, 0xb3,
|
0x07, 0x83, 0xac, 0x43, 0x35, 0x48, 0x7f, 0x3b, 0xcd, 0x4f, 0x8b, 0x5a, 0x4a, 0xfb, 0xb9, 0xf9,
|
||||||
0xff, 0x4a, 0x08, 0x40, 0xe9, 0xc0, 0xe6, 0x02, 0x99, 0x7e, 0x47, 0x7e, 0x9b, 0x68, 0xbb, 0xc8,
|
0x59, 0x2a, 0xa5, 0x95, 0x6d, 0xfe, 0x56, 0x24, 0x4d, 0xa8, 0x05, 0x2a, 0xa5, 0x0f, 0xa8, 0x17,
|
||||||
0x74, 0x4d, 0x7e, 0x7f, 0xc8, 0xa8, 0xd4, 0xe7, 0x24, 0xa5, 0x47, 0x32, 0x75, 0xf5, 0xfc, 0xd3,
|
0x98, 0x5f, 0xd7, 0xee, 0xdc, 0x83, 0xb5, 0xd9, 0x7f, 0x25, 0x04, 0xa0, 0xf2, 0xd8, 0x15, 0x12,
|
||||||
0xde, 0x47, 0x4f, 0x47, 0x54, 0x8c, 0x27, 0x43, 0x59, 0x0c, 0x76, 0xcf, 0xa9, 0xef, 0xd3, 0x73,
|
0xb9, 0x79, 0x4d, 0x7d, 0xdb, 0xe8, 0xfa, 0xc8, 0x4d, 0x43, 0x7d, 0xbf, 0xcf, 0xa9, 0xd2, 0x17,
|
||||||
0x81, 0xce, 0x78, 0x37, 0x89, 0x8c, 0xb7, 0x5d, 0xca, 0x05, 0xa3, 0xc3, 0x89, 0x40, 0x77, 0x77,
|
0x14, 0xa5, 0x4f, 0x55, 0xe9, 0x9a, 0xc5, 0xfb, 0xbd, 0x0f, 0xee, 0x8f, 0xa8, 0x1c, 0x4f, 0x86,
|
||||||
0x16, 0x1f, 0xbb, 0x2a, 0x5c, 0xe6, 0x62, 0x3c, 0x1c, 0x96, 0x94, 0xe6, 0xf1, 0x5f, 0x01, 0x00,
|
0xaa, 0x19, 0xec, 0x9f, 0xd0, 0x30, 0xa4, 0x27, 0x12, 0xbd, 0xf1, 0x7e, 0x9a, 0x19, 0x6f, 0xfa,
|
||||||
0x00, 0xff, 0xff, 0x7a, 0xb3, 0x5a, 0xe2, 0x4b, 0x12, 0x00, 0x00,
|
0x54, 0x48, 0x4e, 0x87, 0x13, 0x89, 0xfe, 0xfe, 0x2c, 0x3f, 0xf6, 0x75, 0xba, 0xcc, 0xc5, 0x64,
|
||||||
|
0x38, 0xac, 0x68, 0xcd, 0xdd, 0x3f, 0x02, 0x00, 0x00, 0xff, 0xff, 0x2d, 0x19, 0x41, 0x9d, 0x25,
|
||||||
|
0x13, 0x00, 0x00,
|
||||||
}
|
}
|
||||||
|
|||||||
@ -89,5 +89,6 @@ service Master {
|
|||||||
rpc ShowPartitions(internal.ShowPartitionRequest) returns (service.StringListResponse) {}
|
rpc ShowPartitions(internal.ShowPartitionRequest) returns (service.StringListResponse) {}
|
||||||
|
|
||||||
|
|
||||||
rpc Tso(stream internal.TsoRequest) returns (stream internal.TsoResponse) {}
|
rpc AllocTimestamp(internal.TsoRequest) returns (internal.TsoResponse) {}
|
||||||
|
rpc AllocId(internal.IdRequest) returns (internal.IdResponse) {}
|
||||||
}
|
}
|
||||||
|
|||||||
@ -30,33 +30,34 @@ const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
|
|||||||
func init() { proto.RegisterFile("master.proto", fileDescriptor_f9c348dec43a6705) }
|
func init() { proto.RegisterFile("master.proto", fileDescriptor_f9c348dec43a6705) }
|
||||||
|
|
||||||
var fileDescriptor_f9c348dec43a6705 = []byte{
|
var fileDescriptor_f9c348dec43a6705 = []byte{
|
||||||
// 409 bytes of a gzipped FileDescriptorProto
|
// 432 bytes of a gzipped FileDescriptorProto
|
||||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x94, 0xcd, 0x6e, 0xe2, 0x30,
|
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x94, 0x51, 0xaf, 0xd2, 0x30,
|
||||||
0x14, 0x85, 0x41, 0x23, 0xa1, 0x91, 0xc5, 0xcf, 0xe0, 0xd9, 0x31, 0xab, 0xc9, 0x0a, 0xc1, 0x90,
|
0x14, 0xc7, 0x79, 0xba, 0xc6, 0x86, 0xcb, 0xf5, 0xd6, 0x37, 0x7c, 0xf1, 0xee, 0xc9, 0x80, 0x6c,
|
||||||
0xa0, 0xe9, 0x0b, 0x54, 0xc0, 0x82, 0x45, 0x2b, 0x55, 0xc0, 0xaa, 0x55, 0x45, 0x9d, 0x60, 0x81,
|
0x46, 0xbf, 0x80, 0x02, 0x0f, 0x90, 0x68, 0x42, 0x80, 0x17, 0x35, 0x06, 0xbb, 0xad, 0x81, 0xc6,
|
||||||
0xd5, 0x24, 0x4e, 0x7d, 0x6f, 0xa8, 0xc4, 0xfb, 0xf6, 0x3d, 0xaa, 0x24, 0x84, 0xc4, 0x05, 0x53,
|
0x6e, 0x9d, 0x3d, 0x67, 0x98, 0xf0, 0xe1, 0xfc, 0x6c, 0x66, 0x1b, 0xdd, 0x56, 0xa1, 0x88, 0xf7,
|
||||||
0xda, 0x1d, 0xb6, 0x8f, 0xbf, 0x83, 0xef, 0x39, 0x0a, 0xa9, 0x07, 0x0c, 0x90, 0x2b, 0x3b, 0x52,
|
0x8d, 0xb6, 0xff, 0xf3, 0xfb, 0x73, 0xce, 0xf9, 0x67, 0xa4, 0x9b, 0x30, 0x40, 0xae, 0xfd, 0x4c,
|
||||||
0x12, 0x25, 0xfd, 0x1d, 0x08, 0x7f, 0x1b, 0x43, 0xb6, 0xb2, 0xb3, 0xa3, 0x4e, 0xdd, 0x93, 0x41,
|
0x2b, 0x54, 0xf4, 0x79, 0x22, 0xe4, 0x3e, 0x87, 0xea, 0xe4, 0x57, 0x4f, 0xfd, 0x6e, 0xa4, 0x92,
|
||||||
0x20, 0xc3, 0x6c, 0xb3, 0x43, 0x45, 0x88, 0x5c, 0x85, 0xcc, 0x5f, 0x06, 0xb0, 0xde, 0xef, 0xb5,
|
0x44, 0xa5, 0xd5, 0x65, 0x9f, 0x8a, 0x14, 0xb9, 0x4e, 0x99, 0xdc, 0x24, 0xb0, 0x3d, 0xde, 0xdd,
|
||||||
0x81, 0xab, 0xad, 0xf0, 0x78, 0xb1, 0xf5, 0xff, 0xed, 0x27, 0xa9, 0xdd, 0xa6, 0xf7, 0x29, 0x23,
|
0x03, 0xd7, 0x7b, 0x11, 0xf1, 0xe6, 0xea, 0xed, 0xef, 0xa7, 0xe4, 0xe6, 0x53, 0x59, 0x4f, 0x19,
|
||||||
0xbf, 0xc6, 0x8a, 0x33, 0xe4, 0x63, 0xe9, 0xfb, 0xdc, 0x43, 0x21, 0x43, 0x6a, 0xdb, 0x9a, 0x53,
|
0x79, 0x36, 0xd1, 0x9c, 0x21, 0x9f, 0x28, 0x29, 0x79, 0x84, 0x42, 0xa5, 0xd4, 0xf7, 0x2d, 0x27,
|
||||||
0xce, 0xb4, 0x3f, 0x0a, 0x67, 0xfc, 0x25, 0xe6, 0x80, 0x9d, 0x3f, 0xba, 0x7e, 0xff, 0x8f, 0xe6,
|
0xc3, 0xf4, 0xff, 0x16, 0x2e, 0xf9, 0xcf, 0x9c, 0x03, 0xf6, 0x5f, 0xd8, 0xfa, 0xe3, 0x3f, 0x5a,
|
||||||
0xc8, 0x30, 0x06, 0xab, 0x42, 0x1f, 0x49, 0x73, 0xa2, 0x64, 0x54, 0x32, 0xf8, 0x67, 0x30, 0xd0,
|
0x21, 0xc3, 0x1c, 0xbc, 0x0e, 0xfd, 0x46, 0x7a, 0x53, 0xad, 0xb2, 0x96, 0xc1, 0x6b, 0x87, 0x81,
|
||||||
0x65, 0x17, 0xe2, 0x5d, 0xd2, 0x98, 0x32, 0x28, 0xd1, 0xfb, 0x06, 0xba, 0xa6, 0xca, 0xe1, 0x96,
|
0x2d, 0xbb, 0x12, 0x1f, 0x92, 0xdb, 0x19, 0x83, 0x16, 0x7d, 0xe8, 0xa0, 0x5b, 0x2a, 0x03, 0xf7,
|
||||||
0x2e, 0xde, 0xcf, 0xca, 0x1e, 0x49, 0xe9, 0xcf, 0x38, 0x44, 0x32, 0x04, 0x6e, 0x55, 0x68, 0x4c,
|
0x6c, 0xf1, 0x71, 0x56, 0xfe, 0x58, 0x29, 0xb9, 0xe4, 0x90, 0xa9, 0x14, 0xb8, 0xd7, 0xa1, 0x39,
|
||||||
0xe8, 0x84, 0x83, 0xa7, 0x84, 0x5b, 0x9e, 0xd3, 0xd0, 0xf4, 0x8c, 0x23, 0x69, 0xee, 0xd6, 0x3f,
|
0xa1, 0x53, 0x0e, 0x91, 0x16, 0x61, 0x7b, 0x4e, 0x6f, 0x5c, 0x6d, 0x9c, 0x48, 0x8d, 0xdb, 0xf0,
|
||||||
0xed, 0x56, 0x08, 0xb3, 0xab, 0x51, 0xf2, 0xd3, 0xaa, 0xd0, 0x67, 0xd2, 0x9a, 0x6f, 0xe4, 0x6b,
|
0xbc, 0x5b, 0x23, 0xac, 0x4a, 0xb3, 0xe2, 0xa7, 0xd7, 0xa1, 0x3f, 0xc8, 0xdd, 0x6a, 0xa7, 0x7e,
|
||||||
0x71, 0x0c, 0xc6, 0xd1, 0xe9, 0xba, 0xdc, 0xaf, 0x7b, 0xda, 0x6f, 0x8e, 0x4a, 0x84, 0xeb, 0x1b,
|
0x35, 0xcf, 0xe0, 0x1c, 0x9d, 0xad, 0x33, 0x7e, 0xaf, 0xce, 0xfb, 0xad, 0x50, 0x8b, 0x74, 0xfb,
|
||||||
0x01, 0x58, 0x7a, 0xe3, 0x92, 0xb4, 0xb2, 0x80, 0xef, 0x98, 0x42, 0x91, 0x3e, 0x70, 0x70, 0xb6,
|
0x51, 0x00, 0xb6, 0x7a, 0xdc, 0x90, 0xbb, 0x6a, 0xc1, 0x0b, 0xa6, 0x51, 0x94, 0x0d, 0x8e, 0x2e,
|
||||||
0x08, 0x07, 0xdd, 0x85, 0x41, 0x3d, 0x90, 0x46, 0x12, 0x70, 0x81, 0xef, 0x9f, 0xa9, 0xc1, 0x57,
|
0x06, 0xa1, 0xd6, 0x5d, 0xb9, 0xa8, 0xaf, 0xe4, 0xb6, 0x58, 0x70, 0x83, 0x1f, 0x5e, 0x88, 0xc1,
|
||||||
0xe1, 0x4f, 0xa4, 0x3e, 0x65, 0x50, 0xb0, 0x7b, 0xe6, 0x12, 0x1c, 0xa1, 0x2f, 0xeb, 0x80, 0x22,
|
0xff, 0xc2, 0xbf, 0x93, 0xee, 0x8c, 0x41, 0xc3, 0x1e, 0xb8, 0x43, 0x70, 0x82, 0xbe, 0x2e, 0x03,
|
||||||
0xed, 0x3c, 0xd8, 0xc2, 0xc6, 0xf9, 0xa4, 0x02, 0x47, 0x5e, 0xbd, 0xd3, 0x5e, 0x07, 0x9d, 0x5e,
|
0x9a, 0xdc, 0x9b, 0xc5, 0x36, 0x36, 0xc1, 0x3f, 0x22, 0x70, 0xe2, 0x35, 0x38, 0xef, 0x55, 0xeb,
|
||||||
0x00, 0x41, 0x9a, 0x49, 0xb0, 0x87, 0x53, 0x30, 0xce, 0x4c, 0x93, 0x7d, 0x27, 0xfe, 0x19, 0xf9,
|
0xec, 0x00, 0x08, 0xd2, 0x2b, 0x16, 0x5b, 0xbf, 0x82, 0x73, 0x66, 0x96, 0xec, 0x31, 0xeb, 0xff,
|
||||||
0xb1, 0x00, 0x49, 0xff, 0x1a, 0xf8, 0x0b, 0x90, 0x86, 0x71, 0xe9, 0x92, 0x9c, 0xd7, 0xad, 0x0e,
|
0x4c, 0x7a, 0x1f, 0xa4, 0x54, 0xd1, 0x5a, 0x24, 0x1c, 0x90, 0x25, 0x19, 0x7d, 0x70, 0x58, 0xad,
|
||||||
0xab, 0xa3, 0xd1, 0xfd, 0xf5, 0x5a, 0xe0, 0x26, 0x76, 0x93, 0xb8, 0x9c, 0x9d, 0xf0, 0x7d, 0xb1,
|
0x41, 0x39, 0x26, 0x67, 0x4b, 0x6a, 0xf4, 0x82, 0x3c, 0x29, 0xd1, 0xf3, 0x98, 0xbe, 0x74, 0x14,
|
||||||
0x43, 0xee, 0x6d, 0x9c, 0x0c, 0x30, 0x58, 0x09, 0x40, 0x25, 0xdc, 0x18, 0xf9, 0xca, 0xc9, 0x31,
|
0xcc, 0x63, 0x83, 0x7c, 0xb8, 0xa0, 0x30, 0xc4, 0xf1, 0xf8, 0xcb, 0xfb, 0xad, 0xc0, 0x5d, 0x1e,
|
||||||
0x4e, 0x4a, 0x75, 0xb2, 0xcf, 0x5b, 0xe4, 0xba, 0xb5, 0x74, 0x7d, 0xf5, 0x1e, 0x00, 0x00, 0xff,
|
0x16, 0x39, 0x08, 0x0e, 0x42, 0x4a, 0x71, 0x40, 0x1e, 0xed, 0x82, 0xaa, 0x76, 0x14, 0x0b, 0x40,
|
||||||
0xff, 0x47, 0x62, 0x7a, 0xee, 0x0c, 0x05, 0x00, 0x00,
|
0x2d, 0xc2, 0x1c, 0x79, 0x1c, 0x18, 0x42, 0x50, 0x02, 0x83, 0xea, 0xbb, 0x99, 0x85, 0xe1, 0x4d,
|
||||||
|
0x79, 0x7e, 0xf7, 0x27, 0x00, 0x00, 0xff, 0xff, 0x3f, 0x05, 0x0f, 0x90, 0x65, 0x05, 0x00, 0x00,
|
||||||
}
|
}
|
||||||
|
|
||||||
// Reference imports to suppress errors if they are not otherwise used.
|
// Reference imports to suppress errors if they are not otherwise used.
|
||||||
@ -131,7 +132,8 @@ type MasterClient interface {
|
|||||||
//
|
//
|
||||||
// @return StringListResponse
|
// @return StringListResponse
|
||||||
ShowPartitions(ctx context.Context, in *internalpb.ShowPartitionRequest, opts ...grpc.CallOption) (*servicepb.StringListResponse, error)
|
ShowPartitions(ctx context.Context, in *internalpb.ShowPartitionRequest, opts ...grpc.CallOption) (*servicepb.StringListResponse, error)
|
||||||
Tso(ctx context.Context, opts ...grpc.CallOption) (Master_TsoClient, error)
|
AllocTimestamp(ctx context.Context, in *internalpb.TsoRequest, opts ...grpc.CallOption) (*internalpb.TsoResponse, error)
|
||||||
|
AllocId(ctx context.Context, in *internalpb.IdRequest, opts ...grpc.CallOption) (*internalpb.IdResponse, error)
|
||||||
}
|
}
|
||||||
|
|
||||||
type masterClient struct {
|
type masterClient struct {
|
||||||
@ -232,35 +234,22 @@ func (c *masterClient) ShowPartitions(ctx context.Context, in *internalpb.ShowPa
|
|||||||
return out, nil
|
return out, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *masterClient) Tso(ctx context.Context, opts ...grpc.CallOption) (Master_TsoClient, error) {
|
func (c *masterClient) AllocTimestamp(ctx context.Context, in *internalpb.TsoRequest, opts ...grpc.CallOption) (*internalpb.TsoResponse, error) {
|
||||||
stream, err := c.cc.NewStream(ctx, &_Master_serviceDesc.Streams[0], "/milvus.proto.master.Master/Tso", opts...)
|
out := new(internalpb.TsoResponse)
|
||||||
|
err := c.cc.Invoke(ctx, "/milvus.proto.master.Master/AllocTimestamp", in, out, opts...)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
x := &masterTsoClient{stream}
|
return out, nil
|
||||||
return x, nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
type Master_TsoClient interface {
|
func (c *masterClient) AllocId(ctx context.Context, in *internalpb.IdRequest, opts ...grpc.CallOption) (*internalpb.IdResponse, error) {
|
||||||
Send(*internalpb.TsoRequest) error
|
out := new(internalpb.IdResponse)
|
||||||
Recv() (*internalpb.TsoResponse, error)
|
err := c.cc.Invoke(ctx, "/milvus.proto.master.Master/AllocId", in, out, opts...)
|
||||||
grpc.ClientStream
|
if err != nil {
|
||||||
}
|
|
||||||
|
|
||||||
type masterTsoClient struct {
|
|
||||||
grpc.ClientStream
|
|
||||||
}
|
|
||||||
|
|
||||||
func (x *masterTsoClient) Send(m *internalpb.TsoRequest) error {
|
|
||||||
return x.ClientStream.SendMsg(m)
|
|
||||||
}
|
|
||||||
|
|
||||||
func (x *masterTsoClient) Recv() (*internalpb.TsoResponse, error) {
|
|
||||||
m := new(internalpb.TsoResponse)
|
|
||||||
if err := x.ClientStream.RecvMsg(m); err != nil {
|
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
return m, nil
|
return out, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// MasterServer is the server API for Master service.
|
// MasterServer is the server API for Master service.
|
||||||
@ -325,7 +314,8 @@ type MasterServer interface {
|
|||||||
//
|
//
|
||||||
// @return StringListResponse
|
// @return StringListResponse
|
||||||
ShowPartitions(context.Context, *internalpb.ShowPartitionRequest) (*servicepb.StringListResponse, error)
|
ShowPartitions(context.Context, *internalpb.ShowPartitionRequest) (*servicepb.StringListResponse, error)
|
||||||
Tso(Master_TsoServer) error
|
AllocTimestamp(context.Context, *internalpb.TsoRequest) (*internalpb.TsoResponse, error)
|
||||||
|
AllocId(context.Context, *internalpb.IdRequest) (*internalpb.IdResponse, error)
|
||||||
}
|
}
|
||||||
|
|
||||||
// UnimplementedMasterServer can be embedded to have forward compatible implementations.
|
// UnimplementedMasterServer can be embedded to have forward compatible implementations.
|
||||||
@ -362,8 +352,11 @@ func (*UnimplementedMasterServer) DescribePartition(ctx context.Context, req *in
|
|||||||
func (*UnimplementedMasterServer) ShowPartitions(ctx context.Context, req *internalpb.ShowPartitionRequest) (*servicepb.StringListResponse, error) {
|
func (*UnimplementedMasterServer) ShowPartitions(ctx context.Context, req *internalpb.ShowPartitionRequest) (*servicepb.StringListResponse, error) {
|
||||||
return nil, status.Errorf(codes.Unimplemented, "method ShowPartitions not implemented")
|
return nil, status.Errorf(codes.Unimplemented, "method ShowPartitions not implemented")
|
||||||
}
|
}
|
||||||
func (*UnimplementedMasterServer) Tso(srv Master_TsoServer) error {
|
func (*UnimplementedMasterServer) AllocTimestamp(ctx context.Context, req *internalpb.TsoRequest) (*internalpb.TsoResponse, error) {
|
||||||
return status.Errorf(codes.Unimplemented, "method Tso not implemented")
|
return nil, status.Errorf(codes.Unimplemented, "method AllocTimestamp not implemented")
|
||||||
|
}
|
||||||
|
func (*UnimplementedMasterServer) AllocId(ctx context.Context, req *internalpb.IdRequest) (*internalpb.IdResponse, error) {
|
||||||
|
return nil, status.Errorf(codes.Unimplemented, "method AllocId not implemented")
|
||||||
}
|
}
|
||||||
|
|
||||||
func RegisterMasterServer(s *grpc.Server, srv MasterServer) {
|
func RegisterMasterServer(s *grpc.Server, srv MasterServer) {
|
||||||
@ -550,30 +543,40 @@ func _Master_ShowPartitions_Handler(srv interface{}, ctx context.Context, dec fu
|
|||||||
return interceptor(ctx, in, info, handler)
|
return interceptor(ctx, in, info, handler)
|
||||||
}
|
}
|
||||||
|
|
||||||
func _Master_Tso_Handler(srv interface{}, stream grpc.ServerStream) error {
|
func _Master_AllocTimestamp_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||||
return srv.(MasterServer).Tso(&masterTsoServer{stream})
|
in := new(internalpb.TsoRequest)
|
||||||
}
|
if err := dec(in); err != nil {
|
||||||
|
|
||||||
type Master_TsoServer interface {
|
|
||||||
Send(*internalpb.TsoResponse) error
|
|
||||||
Recv() (*internalpb.TsoRequest, error)
|
|
||||||
grpc.ServerStream
|
|
||||||
}
|
|
||||||
|
|
||||||
type masterTsoServer struct {
|
|
||||||
grpc.ServerStream
|
|
||||||
}
|
|
||||||
|
|
||||||
func (x *masterTsoServer) Send(m *internalpb.TsoResponse) error {
|
|
||||||
return x.ServerStream.SendMsg(m)
|
|
||||||
}
|
|
||||||
|
|
||||||
func (x *masterTsoServer) Recv() (*internalpb.TsoRequest, error) {
|
|
||||||
m := new(internalpb.TsoRequest)
|
|
||||||
if err := x.ServerStream.RecvMsg(m); err != nil {
|
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
return m, nil
|
if interceptor == nil {
|
||||||
|
return srv.(MasterServer).AllocTimestamp(ctx, in)
|
||||||
|
}
|
||||||
|
info := &grpc.UnaryServerInfo{
|
||||||
|
Server: srv,
|
||||||
|
FullMethod: "/milvus.proto.master.Master/AllocTimestamp",
|
||||||
|
}
|
||||||
|
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||||
|
return srv.(MasterServer).AllocTimestamp(ctx, req.(*internalpb.TsoRequest))
|
||||||
|
}
|
||||||
|
return interceptor(ctx, in, info, handler)
|
||||||
|
}
|
||||||
|
|
||||||
|
func _Master_AllocId_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||||
|
in := new(internalpb.IdRequest)
|
||||||
|
if err := dec(in); err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
if interceptor == nil {
|
||||||
|
return srv.(MasterServer).AllocId(ctx, in)
|
||||||
|
}
|
||||||
|
info := &grpc.UnaryServerInfo{
|
||||||
|
Server: srv,
|
||||||
|
FullMethod: "/milvus.proto.master.Master/AllocId",
|
||||||
|
}
|
||||||
|
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||||
|
return srv.(MasterServer).AllocId(ctx, req.(*internalpb.IdRequest))
|
||||||
|
}
|
||||||
|
return interceptor(ctx, in, info, handler)
|
||||||
}
|
}
|
||||||
|
|
||||||
var _Master_serviceDesc = grpc.ServiceDesc{
|
var _Master_serviceDesc = grpc.ServiceDesc{
|
||||||
@ -620,14 +623,15 @@ var _Master_serviceDesc = grpc.ServiceDesc{
|
|||||||
MethodName: "ShowPartitions",
|
MethodName: "ShowPartitions",
|
||||||
Handler: _Master_ShowPartitions_Handler,
|
Handler: _Master_ShowPartitions_Handler,
|
||||||
},
|
},
|
||||||
},
|
|
||||||
Streams: []grpc.StreamDesc{
|
|
||||||
{
|
{
|
||||||
StreamName: "Tso",
|
MethodName: "AllocTimestamp",
|
||||||
Handler: _Master_Tso_Handler,
|
Handler: _Master_AllocTimestamp_Handler,
|
||||||
ServerStreams: true,
|
},
|
||||||
ClientStreams: true,
|
{
|
||||||
|
MethodName: "AllocId",
|
||||||
|
Handler: _Master_AllocId_Handler,
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
|
Streams: []grpc.StreamDesc{},
|
||||||
Metadata: "master.proto",
|
Metadata: "master.proto",
|
||||||
}
|
}
|
||||||
|
|||||||
@ -2,14 +2,15 @@ package proxy
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
|
"log"
|
||||||
|
"sync"
|
||||||
|
|
||||||
"github.com/apache/pulsar-client-go/pulsar"
|
"github.com/apache/pulsar-client-go/pulsar"
|
||||||
"github.com/golang/protobuf/proto"
|
"github.com/golang/protobuf/proto"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||||
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||||
"log"
|
|
||||||
"sync"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
type manipulationReq struct {
|
type manipulationReq struct {
|
||||||
|
|||||||
59
internal/proxy/manipulation_task.go
Normal file
59
internal/proxy/manipulation_task.go
Normal 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
|
||||||
|
}
|
||||||
94
internal/proxy/proxy_instance.go
Normal file
94
internal/proxy/proxy_instance.go
Normal 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
|
||||||
|
}
|
||||||
@ -3,6 +3,8 @@ package proxy
|
|||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"fmt"
|
"fmt"
|
||||||
|
"strconv"
|
||||||
|
|
||||||
"github.com/apache/pulsar-client-go/pulsar"
|
"github.com/apache/pulsar-client-go/pulsar"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/allocator"
|
"github.com/zilliztech/milvus-distributed/internal/allocator"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||||
@ -10,7 +12,6 @@ import (
|
|||||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||||
etcd "go.etcd.io/etcd/clientv3"
|
etcd "go.etcd.io/etcd/clientv3"
|
||||||
"strconv"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
type BaseRequest interface {
|
type BaseRequest interface {
|
||||||
@ -100,7 +101,7 @@ func StartProxy(opt *ProxyOptions) error {
|
|||||||
// saveInterval: opt.tsoSaveInterval,
|
// saveInterval: opt.tsoSaveInterval,
|
||||||
//}
|
//}
|
||||||
//tso.Restart(opt.proxyId)
|
//tso.Restart(opt.proxyId)
|
||||||
tso := allocator.NewTimestampAllocator()
|
tso, _ := allocator.NewTimestampAllocator(opt.ctx)
|
||||||
|
|
||||||
/////////////////// proxy server ///////////////////////////////
|
/////////////////// proxy server ///////////////////////////////
|
||||||
//readerTopics, send insert and delete message into these topics
|
//readerTopics, send insert and delete message into these topics
|
||||||
|
|||||||
@ -4,27 +4,35 @@ import (
|
|||||||
"context"
|
"context"
|
||||||
"encoding/binary"
|
"encoding/binary"
|
||||||
"encoding/json"
|
"encoding/json"
|
||||||
"github.com/apache/pulsar-client-go/pulsar"
|
"os"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/allocator"
|
|
||||||
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
|
||||||
"github.com/golang/protobuf/proto"
|
|
||||||
"github.com/stretchr/testify/assert"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
|
|
||||||
etcd "go.etcd.io/etcd/clientv3"
|
|
||||||
"google.golang.org/grpc"
|
|
||||||
"sort"
|
"sort"
|
||||||
"strconv"
|
"strconv"
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
)
|
|
||||||
|
|
||||||
|
"github.com/apache/pulsar-client-go/pulsar"
|
||||||
|
"github.com/golang/protobuf/proto"
|
||||||
|
"github.com/stretchr/testify/assert"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/allocator"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
|
||||||
|
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
|
||||||
|
etcd "go.etcd.io/etcd/clientv3"
|
||||||
|
"google.golang.org/grpc"
|
||||||
|
)
|
||||||
|
|
||||||
const (
|
const (
|
||||||
tsoKeyPath string = "/timestampOracle"
|
tsoKeyPath string = "/timestampOracle"
|
||||||
)
|
)
|
||||||
|
|
||||||
var timeAllocator *allocator.TimestampAllocator = allocator.NewTimestampAllocator()
|
var timeAllocator *allocator.TimestampAllocator
|
||||||
|
|
||||||
|
func TestMain(m *testing.M) {
|
||||||
|
timeAllocator, _ = allocator.NewTimestampAllocator(context.Background())
|
||||||
|
exitCode := m.Run()
|
||||||
|
timeAllocator.Close()
|
||||||
|
os.Exit(exitCode)
|
||||||
|
}
|
||||||
|
|
||||||
func TestProxyNode(t *testing.T) {
|
func TestProxyNode(t *testing.T) {
|
||||||
startTestMaster("localhost:11000", t)
|
startTestMaster("localhost:11000", t)
|
||||||
|
|||||||
@ -1,13 +1,14 @@
|
|||||||
package proxy
|
package proxy
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"log"
|
||||||
|
"sync"
|
||||||
|
|
||||||
"github.com/apache/pulsar-client-go/pulsar"
|
"github.com/apache/pulsar-client-go/pulsar"
|
||||||
"github.com/golang/protobuf/proto"
|
"github.com/golang/protobuf/proto"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
|
||||||
"log"
|
|
||||||
"sync"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
type queryReq struct {
|
type queryReq struct {
|
||||||
@ -80,7 +81,6 @@ func (s *proxyServer) restartQueryRoutine(buf_size int) error {
|
|||||||
}
|
}
|
||||||
qm.Timestamp = uint64(ts[0])
|
qm.Timestamp = uint64(ts[0])
|
||||||
|
|
||||||
|
|
||||||
qb, err := proto.Marshal(qm)
|
qb, err := proto.Marshal(qm)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Printf("Marshal QueryReqMsg failed, error = %v", err)
|
log.Printf("Marshal QueryReqMsg failed, error = %v", err)
|
||||||
|
|||||||
@ -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
|
|
||||||
}
|
|
||||||
}
|
|
||||||
@ -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)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
@ -1,8 +1,9 @@
|
|||||||
package proxy
|
package proxy
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
|
||||||
"sync"
|
"sync"
|
||||||
|
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||||
)
|
)
|
||||||
|
|
||||||
type requestScheduler struct {
|
type requestScheduler struct {
|
||||||
|
|||||||
@ -4,6 +4,12 @@ import (
|
|||||||
"context"
|
"context"
|
||||||
"encoding/binary"
|
"encoding/binary"
|
||||||
"encoding/json"
|
"encoding/json"
|
||||||
|
"net"
|
||||||
|
"sort"
|
||||||
|
"testing"
|
||||||
|
"time"
|
||||||
|
"unsafe"
|
||||||
|
|
||||||
"github.com/apache/pulsar-client-go/pulsar"
|
"github.com/apache/pulsar-client-go/pulsar"
|
||||||
"github.com/golang/protobuf/proto"
|
"github.com/golang/protobuf/proto"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
@ -13,11 +19,6 @@ import (
|
|||||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||||
"go.etcd.io/etcd/clientv3"
|
"go.etcd.io/etcd/clientv3"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
"net"
|
|
||||||
"sort"
|
|
||||||
"testing"
|
|
||||||
"time"
|
|
||||||
"unsafe"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
type testMasterServer struct {
|
type testMasterServer struct {
|
||||||
|
|||||||
43
internal/proxy/task.go
Normal file
43
internal/proxy/task.go
Normal 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
|
||||||
|
}
|
||||||
227
internal/proxy/task_scheduler.go
Normal file
227
internal/proxy/task_scheduler.go
Normal 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
|
||||||
|
}
|
||||||
@ -2,13 +2,14 @@ package proxy
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"github.com/apache/pulsar-client-go/pulsar"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
|
||||||
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
|
||||||
"github.com/golang/protobuf/proto"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
|
||||||
"log"
|
"log"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/apache/pulsar-client-go/pulsar"
|
||||||
|
"github.com/golang/protobuf/proto"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||||
|
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||||
)
|
)
|
||||||
|
|
||||||
type timeTick struct {
|
type timeTick struct {
|
||||||
|
|||||||
@ -2,13 +2,14 @@ package proxy
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
"testing"
|
||||||
|
"time"
|
||||||
|
|
||||||
"github.com/apache/pulsar-client-go/pulsar"
|
"github.com/apache/pulsar-client-go/pulsar"
|
||||||
"github.com/golang/protobuf/proto"
|
"github.com/golang/protobuf/proto"
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||||
"testing"
|
|
||||||
"time"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestTimeTick(t *testing.T) {
|
func TestTimeTick(t *testing.T) {
|
||||||
|
|||||||
@ -2,8 +2,9 @@ package reader
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"github.com/stretchr/testify/assert"
|
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
|
"github.com/stretchr/testify/assert"
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestCollection_NewPartition(t *testing.T) {
|
func TestCollection_NewPartition(t *testing.T) {
|
||||||
|
|||||||
@ -3,10 +3,11 @@ package reader
|
|||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"fmt"
|
"fmt"
|
||||||
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/util/flowgraph"
|
|
||||||
"log"
|
"log"
|
||||||
"sync"
|
"sync"
|
||||||
|
|
||||||
|
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/util/flowgraph"
|
||||||
)
|
)
|
||||||
|
|
||||||
type manipulationService struct {
|
type manipulationService struct {
|
||||||
|
|||||||
@ -3,15 +3,16 @@ package reader
|
|||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"encoding/binary"
|
"encoding/binary"
|
||||||
"github.com/stretchr/testify/assert"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/msgclient"
|
|
||||||
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
|
||||||
"math"
|
"math"
|
||||||
"strconv"
|
"strconv"
|
||||||
"sync"
|
"sync"
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/stretchr/testify/assert"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/msgclient"
|
||||||
|
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestInsertAndDelete_MessagesPreprocess(t *testing.T) {
|
func TestInsertAndDelete_MessagesPreprocess(t *testing.T) {
|
||||||
|
|||||||
@ -2,17 +2,18 @@ package reader
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
"log"
|
||||||
|
"math"
|
||||||
|
"sync"
|
||||||
|
"testing"
|
||||||
|
"time"
|
||||||
|
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/master/collection"
|
"github.com/zilliztech/milvus-distributed/internal/master/collection"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/master/segment"
|
"github.com/zilliztech/milvus-distributed/internal/master/segment"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/msgclient"
|
"github.com/zilliztech/milvus-distributed/internal/msgclient"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
||||||
"log"
|
|
||||||
"math"
|
|
||||||
"sync"
|
|
||||||
"testing"
|
|
||||||
"time"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestMeta_GetCollectionObjId(t *testing.T) {
|
func TestMeta_GetCollectionObjId(t *testing.T) {
|
||||||
|
|||||||
@ -2,8 +2,9 @@ package reader
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"github.com/stretchr/testify/assert"
|
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
|
"github.com/stretchr/testify/assert"
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestPartition_NewSegment(t *testing.T) {
|
func TestPartition_NewSegment(t *testing.T) {
|
||||||
|
|||||||
@ -1,8 +1,9 @@
|
|||||||
package reader
|
package reader
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"github.com/stretchr/testify/assert"
|
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
|
"github.com/stretchr/testify/assert"
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestQueryNodeTime_UpdateReadTimeSync(t *testing.T) {
|
func TestQueryNodeTime_UpdateReadTimeSync(t *testing.T) {
|
||||||
|
|||||||
@ -2,9 +2,10 @@ package reader
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
"testing"
|
||||||
|
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||||
"testing"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestQueryNode_CreateQueryNode(t *testing.T) {
|
func TestQueryNode_CreateQueryNode(t *testing.T) {
|
||||||
|
|||||||
@ -3,10 +3,11 @@ package reader
|
|||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"fmt"
|
"fmt"
|
||||||
"github.com/stretchr/testify/assert"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/msgclient"
|
|
||||||
"log"
|
"log"
|
||||||
"sync"
|
"sync"
|
||||||
|
|
||||||
|
"github.com/stretchr/testify/assert"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/msgclient"
|
||||||
)
|
)
|
||||||
|
|
||||||
func StartQueryNode(ctx context.Context, pulsarURL string) {
|
func StartQueryNode(ctx context.Context, pulsarURL string) {
|
||||||
|
|||||||
@ -2,13 +2,14 @@ package reader
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"github.com/stretchr/testify/assert"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/msgclient"
|
|
||||||
"strconv"
|
"strconv"
|
||||||
"sync"
|
"sync"
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/stretchr/testify/assert"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/msgclient"
|
||||||
)
|
)
|
||||||
|
|
||||||
const ctxTimeInMillisecond = 10
|
const ctxTimeInMillisecond = 10
|
||||||
|
|||||||
@ -2,8 +2,9 @@ package reader
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||||
msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -23,8 +24,7 @@ func (node *QueryNode) PublishSearchResult(results *msgpb.QueryResult) commonpb.
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (node *QueryNode) PublishFailedSearchResult() commonpb.Status {
|
func (node *QueryNode) PublishFailedSearchResult() commonpb.Status {
|
||||||
var results = msgpb.QueryResult{
|
var results = msgpb.QueryResult{}
|
||||||
}
|
|
||||||
|
|
||||||
var ctx = context.Background()
|
var ctx = context.Background()
|
||||||
|
|
||||||
|
|||||||
@ -2,13 +2,14 @@ package reader
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"github.com/stretchr/testify/assert"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/msgclient"
|
|
||||||
"strconv"
|
"strconv"
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/stretchr/testify/assert"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/msgclient"
|
||||||
|
|
||||||
//masterPb "github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
|
//masterPb "github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||||
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||||
|
|||||||
@ -2,9 +2,10 @@ package reader
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
|
"sort"
|
||||||
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||||
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||||
"sort"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
func (node *QueryNode) Search(searchMessages []*msgPb.SearchMsg) msgPb.Status {
|
func (node *QueryNode) Search(searchMessages []*msgPb.SearchMsg) msgPb.Status {
|
||||||
|
|||||||
@ -3,14 +3,15 @@ package reader
|
|||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"encoding/binary"
|
"encoding/binary"
|
||||||
"github.com/stretchr/testify/assert"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/msgclient"
|
|
||||||
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
|
||||||
"math"
|
"math"
|
||||||
"strconv"
|
"strconv"
|
||||||
"sync"
|
"sync"
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
|
"github.com/stretchr/testify/assert"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/msgclient"
|
||||||
|
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||||
)
|
)
|
||||||
|
|
||||||
// NOTE: start pulsar before test
|
// NOTE: start pulsar before test
|
||||||
|
|||||||
@ -13,11 +13,12 @@ package reader
|
|||||||
*/
|
*/
|
||||||
import "C"
|
import "C"
|
||||||
import (
|
import (
|
||||||
|
"strconv"
|
||||||
|
"unsafe"
|
||||||
|
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||||
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||||
"strconv"
|
|
||||||
"unsafe"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
const SegmentLifetime = 20000
|
const SegmentLifetime = 20000
|
||||||
|
|||||||
@ -2,11 +2,12 @@ package reader
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
|
||||||
"log"
|
"log"
|
||||||
"strconv"
|
"strconv"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||||
)
|
)
|
||||||
|
|
||||||
//func (node *QueryNode) SegmentsManagement() {
|
//func (node *QueryNode) SegmentsManagement() {
|
||||||
|
|||||||
@ -2,11 +2,12 @@ package reader
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/msgclient"
|
|
||||||
"strconv"
|
"strconv"
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/msgclient"
|
||||||
)
|
)
|
||||||
|
|
||||||
//func TestSegmentManagement_SegmentsManagement(t *testing.T) {
|
//func TestSegmentManagement_SegmentsManagement(t *testing.T) {
|
||||||
|
|||||||
@ -4,10 +4,11 @@ import (
|
|||||||
"encoding/json"
|
"encoding/json"
|
||||||
"errors"
|
"errors"
|
||||||
"fmt"
|
"fmt"
|
||||||
log "github.com/apache/pulsar/pulsar-client-go/logutil"
|
|
||||||
"os"
|
"os"
|
||||||
"strconv"
|
"strconv"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
log "github.com/apache/pulsar/pulsar-client-go/logutil"
|
||||||
)
|
)
|
||||||
|
|
||||||
// Function `GetSegmentByEntityId` should return entityIDs, timestamps and segmentIDs
|
// Function `GetSegmentByEntityId` should return entityIDs, timestamps and segmentIDs
|
||||||
|
|||||||
@ -2,13 +2,14 @@ package reader
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/msgclient"
|
|
||||||
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
|
||||||
"strconv"
|
"strconv"
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/msgclient"
|
||||||
|
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||||
|
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|||||||
@ -2,15 +2,15 @@ package S3_driver_test
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
s3_driver "github.com/zilliztech/milvus-distributed/internal/storage/internal/S3"
|
|
||||||
"github.com/stretchr/testify/assert"
|
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
|
"github.com/stretchr/testify/assert"
|
||||||
|
s3_driver "github.com/zilliztech/milvus-distributed/internal/storage/internal/S3"
|
||||||
)
|
)
|
||||||
|
|
||||||
var ctx = context.Background()
|
var ctx = context.Background()
|
||||||
var client, err = s3_driver.NewS3Driver(ctx)
|
var client, err = s3_driver.NewS3Driver(ctx)
|
||||||
|
|
||||||
|
|
||||||
func TestS3Driver_PutRowAndGetRow(t *testing.T) {
|
func TestS3Driver_PutRowAndGetRow(t *testing.T) {
|
||||||
err = client.PutRow(ctx, []byte("bar"), []byte("abcdefghijklmnoopqrstuvwxyz"), "SegmentA", 1)
|
err = client.PutRow(ctx, []byte("bar"), []byte("abcdefghijklmnoopqrstuvwxyz"), "SegmentA", 1)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|||||||
@ -3,12 +3,13 @@ package S3_driver
|
|||||||
import (
|
import (
|
||||||
"bytes"
|
"bytes"
|
||||||
"context"
|
"context"
|
||||||
|
"io"
|
||||||
|
|
||||||
"github.com/aws/aws-sdk-go/aws"
|
"github.com/aws/aws-sdk-go/aws"
|
||||||
"github.com/aws/aws-sdk-go/aws/session"
|
"github.com/aws/aws-sdk-go/aws/session"
|
||||||
"github.com/aws/aws-sdk-go/service/s3"
|
"github.com/aws/aws-sdk-go/service/s3"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||||
. "github.com/zilliztech/milvus-distributed/internal/storage/type"
|
. "github.com/zilliztech/milvus-distributed/internal/storage/type"
|
||||||
"io"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
var bucketName = conf.Config.Writer.Bucket
|
var bucketName = conf.Config.Writer.Bucket
|
||||||
|
|||||||
@ -2,6 +2,7 @@ package S3_driver
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
|
||||||
"github.com/aws/aws-sdk-go/aws"
|
"github.com/aws/aws-sdk-go/aws"
|
||||||
"github.com/aws/aws-sdk-go/aws/endpoints"
|
"github.com/aws/aws-sdk-go/aws/endpoints"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/storage/internal/minio/codec"
|
"github.com/zilliztech/milvus-distributed/internal/storage/internal/minio/codec"
|
||||||
|
|||||||
@ -2,11 +2,12 @@ package minio_driver
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
|
||||||
|
"github.com/minio/minio-go/v7"
|
||||||
|
"github.com/minio/minio-go/v7/pkg/credentials"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/storage/internal/minio/codec"
|
"github.com/zilliztech/milvus-distributed/internal/storage/internal/minio/codec"
|
||||||
. "github.com/zilliztech/milvus-distributed/internal/storage/internal/minio/codec"
|
. "github.com/zilliztech/milvus-distributed/internal/storage/internal/minio/codec"
|
||||||
. "github.com/zilliztech/milvus-distributed/internal/storage/type"
|
. "github.com/zilliztech/milvus-distributed/internal/storage/type"
|
||||||
"github.com/minio/minio-go/v7"
|
|
||||||
"github.com/minio/minio-go/v7/pkg/credentials"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
type minioDriver struct {
|
type minioDriver struct {
|
||||||
|
|||||||
@ -3,10 +3,11 @@ package minio_driver
|
|||||||
import (
|
import (
|
||||||
"bytes"
|
"bytes"
|
||||||
"context"
|
"context"
|
||||||
|
"io"
|
||||||
|
|
||||||
|
"github.com/minio/minio-go/v7"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||||
. "github.com/zilliztech/milvus-distributed/internal/storage/type"
|
. "github.com/zilliztech/milvus-distributed/internal/storage/type"
|
||||||
"github.com/minio/minio-go/v7"
|
|
||||||
"io"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
var bucketName = conf.Config.Writer.Bucket
|
var bucketName = conf.Config.Writer.Bucket
|
||||||
|
|||||||
@ -2,15 +2,15 @@ package minio_driver_test
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
minio_driver "github.com/zilliztech/milvus-distributed/internal/storage/internal/minio"
|
|
||||||
"github.com/stretchr/testify/assert"
|
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
|
"github.com/stretchr/testify/assert"
|
||||||
|
minio_driver "github.com/zilliztech/milvus-distributed/internal/storage/internal/minio"
|
||||||
)
|
)
|
||||||
|
|
||||||
var ctx = context.Background()
|
var ctx = context.Background()
|
||||||
var client, err = minio_driver.NewMinioDriver(ctx)
|
var client, err = minio_driver.NewMinioDriver(ctx)
|
||||||
|
|
||||||
|
|
||||||
func TestMinioDriver_PutRowAndGetRow(t *testing.T) {
|
func TestMinioDriver_PutRowAndGetRow(t *testing.T) {
|
||||||
err = client.PutRow(ctx, []byte("bar"), []byte("abcdefghijklmnoopqrstuvwxyz"), "SegmentA", 1)
|
err = client.PutRow(ctx, []byte("bar"), []byte("abcdefghijklmnoopqrstuvwxyz"), "SegmentA", 1)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|||||||
@ -3,6 +3,7 @@ package codec
|
|||||||
import (
|
import (
|
||||||
"encoding/binary"
|
"encoding/binary"
|
||||||
"errors"
|
"errors"
|
||||||
|
|
||||||
"github.com/tikv/client-go/codec"
|
"github.com/tikv/client-go/codec"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|||||||
@ -3,13 +3,14 @@ package tikv_driver
|
|||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"errors"
|
"errors"
|
||||||
|
"strconv"
|
||||||
|
"strings"
|
||||||
|
|
||||||
|
"github.com/tikv/client-go/config"
|
||||||
|
"github.com/tikv/client-go/rawkv"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||||
. "github.com/zilliztech/milvus-distributed/internal/storage/internal/tikv/codec"
|
. "github.com/zilliztech/milvus-distributed/internal/storage/internal/tikv/codec"
|
||||||
. "github.com/zilliztech/milvus-distributed/internal/storage/type"
|
. "github.com/zilliztech/milvus-distributed/internal/storage/type"
|
||||||
"github.com/tikv/client-go/config"
|
|
||||||
"github.com/tikv/client-go/rawkv"
|
|
||||||
"strconv"
|
|
||||||
"strings"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
func keyAddOne(key Key) Key {
|
func keyAddOne(key Key) Key {
|
||||||
|
|||||||
@ -4,14 +4,15 @@ import (
|
|||||||
"bytes"
|
"bytes"
|
||||||
"context"
|
"context"
|
||||||
"fmt"
|
"fmt"
|
||||||
. "github.com/zilliztech/milvus-distributed/internal/storage/internal/tikv/codec"
|
|
||||||
. "github.com/zilliztech/milvus-distributed/internal/storage/type"
|
|
||||||
"github.com/stretchr/testify/assert"
|
|
||||||
"github.com/stretchr/testify/require"
|
|
||||||
"math"
|
"math"
|
||||||
"os"
|
"os"
|
||||||
"sort"
|
"sort"
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
|
"github.com/stretchr/testify/assert"
|
||||||
|
"github.com/stretchr/testify/require"
|
||||||
|
. "github.com/zilliztech/milvus-distributed/internal/storage/internal/tikv/codec"
|
||||||
|
. "github.com/zilliztech/milvus-distributed/internal/storage/type"
|
||||||
)
|
)
|
||||||
|
|
||||||
//var store TikvStore
|
//var store TikvStore
|
||||||
|
|||||||
@ -3,10 +3,11 @@ package storage
|
|||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"errors"
|
"errors"
|
||||||
|
|
||||||
S3Driver "github.com/zilliztech/milvus-distributed/internal/storage/internal/S3"
|
S3Driver "github.com/zilliztech/milvus-distributed/internal/storage/internal/S3"
|
||||||
minIODriver "github.com/zilliztech/milvus-distributed/internal/storage/internal/minio"
|
minIODriver "github.com/zilliztech/milvus-distributed/internal/storage/internal/minio"
|
||||||
tikvDriver "github.com/zilliztech/milvus-distributed/internal/storage/internal/tikv"
|
tikvDriver "github.com/zilliztech/milvus-distributed/internal/storage/internal/tikv"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/storage/type"
|
storagetype "github.com/zilliztech/milvus-distributed/internal/storage/type"
|
||||||
)
|
)
|
||||||
|
|
||||||
func NewStore(ctx context.Context, driver storagetype.DriverType) (storagetype.Store, error) {
|
func NewStore(ctx context.Context, driver storagetype.DriverType) (storagetype.Store, error) {
|
||||||
|
|||||||
@ -12,10 +12,10 @@ import (
|
|||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/apache/pulsar-client-go/pulsar"
|
"github.com/apache/pulsar-client-go/pulsar"
|
||||||
|
"github.com/golang/protobuf/proto"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||||
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||||
"github.com/golang/protobuf/proto"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
type InsertLog struct {
|
type InsertLog struct {
|
||||||
|
|||||||
@ -8,9 +8,9 @@ import (
|
|||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/apache/pulsar-client-go/pulsar"
|
"github.com/apache/pulsar-client-go/pulsar"
|
||||||
|
"github.com/golang/protobuf/proto"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||||
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||||
"github.com/golang/protobuf/proto"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
const (
|
const (
|
||||||
|
|||||||
@ -1,16 +1,3 @@
|
|||||||
// Copyright 2016 TiKV Project Authors.
|
|
||||||
//
|
|
||||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
|
||||||
// you may not use this file except in compliance with the License.
|
|
||||||
// You may obtain a copy of the License at
|
|
||||||
//
|
|
||||||
// http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
//
|
|
||||||
// Unless required by applicable law or agreed to in writing, software
|
|
||||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
// See the License for the specific language governing permissions and
|
|
||||||
// limitations under the License.
|
|
||||||
|
|
||||||
package etcdutil
|
package etcdutil
|
||||||
|
|
||||||
import (
|
import (
|
||||||
@ -19,9 +6,9 @@ import (
|
|||||||
"net/http"
|
"net/http"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
|
||||||
"github.com/gogo/protobuf/proto"
|
"github.com/gogo/protobuf/proto"
|
||||||
"github.com/pingcap/log"
|
"github.com/pingcap/log"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||||
"go.etcd.io/etcd/clientv3"
|
"go.etcd.io/etcd/clientv3"
|
||||||
"go.etcd.io/etcd/etcdserver"
|
"go.etcd.io/etcd/etcdserver"
|
||||||
"go.etcd.io/etcd/pkg/types"
|
"go.etcd.io/etcd/pkg/types"
|
||||||
|
|||||||
@ -1,16 +1,3 @@
|
|||||||
// Copyright 2016 TiKV Project Authors.
|
|
||||||
//
|
|
||||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
|
||||||
// you may not use this file except in compliance with the License.
|
|
||||||
// You may obtain a copy of the License at
|
|
||||||
//
|
|
||||||
// http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
//
|
|
||||||
// Unless required by applicable law or agreed to in writing, software
|
|
||||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
// See the License for the specific language governing permissions and
|
|
||||||
// limitations under the License.
|
|
||||||
|
|
||||||
package etcdutil
|
package etcdutil
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
|||||||
@ -2,9 +2,10 @@ package flowgraph
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
"sync"
|
||||||
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||||
"sync"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
type Timestamp = typeutil.Timestamp
|
type Timestamp = typeutil.Timestamp
|
||||||
|
|||||||
@ -1,16 +1,3 @@
|
|||||||
// Copyright 2019 TiKV Project Authors.
|
|
||||||
//
|
|
||||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
|
||||||
// you may not use this file except in compliance with the License.
|
|
||||||
// You may obtain a copy of the License at
|
|
||||||
//
|
|
||||||
// http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
//
|
|
||||||
// Unless required by applicable law or agreed to in writing, software
|
|
||||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
// See the License for the specific language governing permissions and
|
|
||||||
// limitations under the License.
|
|
||||||
|
|
||||||
package tsoutil
|
package tsoutil
|
||||||
|
|
||||||
import (
|
import (
|
||||||
@ -33,4 +20,3 @@ func ParseTS(ts uint64) (time.Time, uint64) {
|
|||||||
physicalTime := time.Unix(int64(physical/1000), int64(physical)%1000*time.Millisecond.Nanoseconds())
|
physicalTime := time.Unix(int64(physical/1000), int64(physical)%1000*time.Millisecond.Nanoseconds())
|
||||||
return physicalTime, logical
|
return physicalTime, logical
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@ -1,20 +1,8 @@
|
|||||||
// Copyright 2016 TiKV Project Authors.
|
|
||||||
//
|
|
||||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
|
||||||
// you may not use this file except in compliance with the License.
|
|
||||||
// You may obtain a copy of the License at
|
|
||||||
//
|
|
||||||
// http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
//
|
|
||||||
// Unless required by applicable law or agreed to in writing, software
|
|
||||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
// See the License for the specific language governing permissions and
|
|
||||||
// limitations under the License.
|
|
||||||
|
|
||||||
package typeutil
|
package typeutil
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"encoding/binary"
|
"encoding/binary"
|
||||||
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|||||||
@ -2,8 +2,9 @@ package typeutil
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"encoding/binary"
|
"encoding/binary"
|
||||||
"github.com/spaolacci/murmur3"
|
|
||||||
"unsafe"
|
"unsafe"
|
||||||
|
|
||||||
|
"github.com/spaolacci/murmur3"
|
||||||
)
|
)
|
||||||
|
|
||||||
func Hash32Bytes(b []byte) (uint32, error) {
|
func Hash32Bytes(b []byte) (uint32, error) {
|
||||||
@ -20,7 +21,6 @@ func Hash32Uint64(v uint64) (uint32, error) {
|
|||||||
return Hash32Bytes(b)
|
return Hash32Bytes(b)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
func Hash32Int64(v int64) (uint32, error) {
|
func Hash32Int64(v int64) (uint32, error) {
|
||||||
return Hash32Uint64(uint64(v))
|
return Hash32Uint64(uint64(v))
|
||||||
}
|
}
|
||||||
|
|||||||
@ -1,8 +1,10 @@
|
|||||||
package typeutil
|
package typeutil
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"github.com/stretchr/testify/assert"
|
|
||||||
"testing"
|
"testing"
|
||||||
"unsafe"
|
"unsafe"
|
||||||
|
|
||||||
|
"github.com/stretchr/testify/assert"
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestUint64(t *testing.T) {
|
func TestUint64(t *testing.T) {
|
||||||
@ -27,4 +29,3 @@ func TestHash32_Uint64(t *testing.T) {
|
|||||||
t.Log(h2)
|
t.Log(h2)
|
||||||
assert.Equal(t, h, h2)
|
assert.Equal(t, h, h2)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@ -1,16 +1,3 @@
|
|||||||
// Copyright 2016 TiKV Project Authors.
|
|
||||||
//
|
|
||||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
|
||||||
// you may not use this file except in compliance with the License.
|
|
||||||
// You may obtain a copy of the License at
|
|
||||||
//
|
|
||||||
// http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
//
|
|
||||||
// Unless required by applicable law or agreed to in writing, software
|
|
||||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
// See the License for the specific language governing permissions and
|
|
||||||
// limitations under the License.
|
|
||||||
|
|
||||||
package typeutil
|
package typeutil
|
||||||
|
|
||||||
import "time"
|
import "time"
|
||||||
|
|||||||
@ -1,5 +1,4 @@
|
|||||||
package typeutil
|
package typeutil
|
||||||
|
|
||||||
|
|
||||||
type Timestamp = uint64
|
type Timestamp = uint64
|
||||||
type Id = int64
|
type Id = int64
|
||||||
@ -5,16 +5,17 @@ import (
|
|||||||
"encoding/binary"
|
"encoding/binary"
|
||||||
"encoding/json"
|
"encoding/json"
|
||||||
"fmt"
|
"fmt"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
|
||||||
msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/storage"
|
|
||||||
storagetype "github.com/zilliztech/milvus-distributed/internal/storage/type"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/msgclient"
|
|
||||||
"log"
|
"log"
|
||||||
"os"
|
"os"
|
||||||
"strconv"
|
"strconv"
|
||||||
"sync"
|
"sync"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/msgclient"
|
||||||
|
msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/storage"
|
||||||
|
storagetype "github.com/zilliztech/milvus-distributed/internal/storage/type"
|
||||||
)
|
)
|
||||||
|
|
||||||
type SegmentIdInfo struct {
|
type SegmentIdInfo struct {
|
||||||
|
|||||||
@ -2,10 +2,11 @@ package test
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/writer"
|
|
||||||
"sync"
|
"sync"
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
|
msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/writer"
|
||||||
)
|
)
|
||||||
|
|
||||||
func GetInsertMsg(collectionName string, partitionTag string, entityId int64) *msgpb.InsertOrDeleteMsg {
|
func GetInsertMsg(collectionName string, partitionTag string, entityId int64) *msgpb.InsertOrDeleteMsg {
|
||||||
|
|||||||
@ -2,12 +2,13 @@ package test
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"github.com/apache/pulsar-client-go/pulsar"
|
|
||||||
msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
|
||||||
"github.com/gogo/protobuf/proto"
|
|
||||||
"github.com/stretchr/testify/assert"
|
|
||||||
"log"
|
"log"
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
|
"github.com/apache/pulsar-client-go/pulsar"
|
||||||
|
"github.com/gogo/protobuf/proto"
|
||||||
|
"github.com/stretchr/testify/assert"
|
||||||
|
msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestKey2Seg(t *testing.T) {
|
func TestKey2Seg(t *testing.T) {
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user