Refactor proxy

Signed-off-by: zhenshan.cao <zhenshan.cao@zilliz.com>
This commit is contained in:
zhenshan.cao 2021-01-29 09:27:26 +08:00 committed by yefu.chen
parent 7957170220
commit ee0faddb07
35 changed files with 1342 additions and 1164 deletions

View File

@ -173,7 +173,7 @@ unittest: test-cpp test-go
test-go:build-cpp test-go:build-cpp
@echo "Running go unittests..." @echo "Running go unittests..."
@echo "disable go unittest for now, enable it later" @echo "disable go unittest for now, enable it later"
#@(env bash $(PWD)/scripts/run_go_unittest.sh) @(env bash $(PWD)/scripts/run_go_unittest.sh)
test-cpp: build-cpp-with-unittest test-cpp: build-cpp-with-unittest
@echo "Running cpp unittests..." @echo "Running cpp unittests..."

View File

@ -34,8 +34,8 @@ func main() {
cnt := 0 cnt := 0
psc.Params.Init() psc.Params.Init()
log.Printf("proxy service address : %s", psc.Params.NetworkAddress()) log.Printf("proxy service address : %s", psc.Params.ServiceAddress)
proxyService := psc.NewClient(psc.Params.NetworkAddress()) proxyService := psc.NewClient(psc.Params.ServiceAddress)
for cnt = 0; cnt < reTryCnt; cnt++ { for cnt = 0; cnt < reTryCnt; cnt++ {
pxStates, err := proxyService.GetComponentStates() pxStates, err := proxyService.GetComponentStates()

View File

@ -14,7 +14,7 @@ import (
func main() { func main() {
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
svr, err := grpcproxynode.CreateProxyNodeServer() svr, err := grpcproxynode.NewServer()
if err != nil { if err != nil {
log.Print("create server failed", zap.Error(err)) log.Print("create server failed", zap.Error(err))
} }
@ -32,14 +32,10 @@ func main() {
cancel() cancel()
}() }()
if err := svr.Init(); err != nil { if err := svr.Run(); err != nil {
log.Fatal("Init server failed", zap.Error(err)) log.Fatal("Init server failed", zap.Error(err))
} }
if err := svr.Start(); err != nil {
log.Fatal("run server failed", zap.Error(err))
}
<-ctx.Done() <-ctx.Done()
log.Print("Got signal to exit", zap.String("signal", sig.String())) log.Print("Got signal to exit", zap.String("signal", sig.String()))

View File

@ -14,7 +14,7 @@ import (
func main() { func main() {
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
svr, err := grpcproxyservice.CreateProxyServiceServer() svr, err := grpcproxyservice.NewServer()
if err != nil { if err != nil {
log.Print("create server failed", zap.Error(err)) log.Print("create server failed", zap.Error(err))
} }
@ -32,14 +32,10 @@ func main() {
cancel() cancel()
}() }()
if err := svr.Init(); err != nil { if err := svr.Run(); err != nil {
log.Fatal("init server failed", zap.Error(err)) log.Fatal("init server failed", zap.Error(err))
} }
if err := svr.Start(); err != nil {
log.Fatal("run server failed", zap.Error(err))
}
<-ctx.Done() <-ctx.Done()
log.Print("Got signal to exit", zap.String("signal", sig.String())) log.Print("Got signal to exit", zap.String("signal", sig.String()))

View File

@ -65,7 +65,7 @@ func InitProxy(wg *sync.WaitGroup) {
//proxynode.Init() //proxynode.Init()
//fmt.Println("ProxyID is", proxynode.Params.ProxyID()) //fmt.Println("ProxyID is", proxynode.Params.ProxyID())
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
svr, err := proxynode.CreateProxyNodeImpl(ctx) svr, err := proxynode.NewProxyNodeImpl(ctx)
if err != nil { if err != nil {
log.Print("create server failed", zap.Error(err)) log.Print("create server failed", zap.Error(err))
} }

View File

@ -3,192 +3,151 @@ package allocator
import ( import (
"context" "context"
"errors" "errors"
"log"
"sync" "sync"
"time" "time"
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
"google.golang.org/grpc"
) )
const ( const (
maxConcurrentRequests = 10000 maxConcurrentRequests = 10000
) )
type request interface { type Request interface {
Wait() Wait()
Notify(error) Notify(error)
IsValid() bool IsValid() bool
} }
type baseRequest struct { type BaseRequest struct {
done chan error Done chan error
valid bool Valid bool
} }
func (req *baseRequest) Wait() { func (req *BaseRequest) Wait() {
err := <-req.done err := <-req.Done
req.valid = err == nil req.Valid = err == nil
} }
func (req *baseRequest) IsValid() bool { func (req *BaseRequest) IsValid() bool {
return req.valid return req.Valid
} }
func (req *baseRequest) Notify(err error) { func (req *BaseRequest) Notify(err error) {
req.done <- err req.Done <- err
} }
type idRequest struct { type IDRequest struct {
baseRequest BaseRequest
id UniqueID id UniqueID
count uint32 count uint32
} }
type tsoRequest struct { type TSORequest struct {
baseRequest BaseRequest
timestamp Timestamp timestamp Timestamp
count uint32 count uint32
} }
type segRequest struct { type SyncRequest struct {
baseRequest BaseRequest
count uint32
colName string
partitionName string
collID UniqueID
partitionID UniqueID
segInfo map[UniqueID]uint32
channelID int32
timestamp Timestamp
} }
type syncRequest struct { type TickerChan interface {
baseRequest
}
type tickerChan interface {
Chan() <-chan time.Time Chan() <-chan time.Time
Close() Close()
Init() Init()
Reset() Reset()
} }
type emptyTicker struct { type EmptyTicker struct {
tChan <-chan time.Time tChan <-chan time.Time
} }
func (t *emptyTicker) Chan() <-chan time.Time { func (t *EmptyTicker) Chan() <-chan time.Time {
return t.tChan return t.tChan
} }
func (t *emptyTicker) Init() { func (t *EmptyTicker) Init() {
} }
func (t *emptyTicker) Reset() { func (t *EmptyTicker) Reset() {
} }
func (t *emptyTicker) Close() { func (t *EmptyTicker) Close() {
} }
type ticker struct { type Ticker struct {
ticker *time.Ticker ticker *time.Ticker
updateInterval time.Duration // UpdateInterval time.Duration //
} }
func (t *ticker) Init() { func (t *Ticker) Init() {
t.ticker = time.NewTicker(t.updateInterval) t.ticker = time.NewTicker(t.UpdateInterval)
} }
func (t *ticker) Reset() { func (t *Ticker) Reset() {
t.ticker.Reset(t.updateInterval) t.ticker.Reset(t.UpdateInterval)
} }
func (t *ticker) Close() { func (t *Ticker) Close() {
t.ticker.Stop() t.ticker.Stop()
} }
func (t *ticker) Chan() <-chan time.Time { func (t *Ticker) Chan() <-chan time.Time {
return t.ticker.C return t.ticker.C
} }
type Allocator struct { type Allocator struct {
reqs chan request Ctx context.Context
CancelFunc context.CancelFunc
wg sync.WaitGroup wg sync.WaitGroup
ctx context.Context
cancel context.CancelFunc
masterAddress string Reqs chan Request
masterConn *grpc.ClientConn ToDoReqs []Request
masterClient masterpb.MasterServiceClient CanDoReqs []Request
countPerRPC uint32 SyncReqs []Request
toDoReqs []request TChan TickerChan
canDoReqs []request ForceSyncChan chan Request
syncReqs []request
tChan tickerChan SyncFunc func() bool
forceSyncChan chan request ProcessFunc func(req Request) error
syncFunc func() bool CheckSyncFunc func(timeout bool) bool
processFunc func(req request) error PickCanDoFunc func()
checkSyncFunc func(timeout bool) bool
pickCanDoFunc func()
} }
func (ta *Allocator) Start() error { func (ta *Allocator) Start() error {
connectMasterFn := func() error { ta.TChan.Init()
return ta.connectMaster()
}
err := Retry(10, time.Millisecond*200, connectMasterFn)
if err != nil {
panic("connect to master failed")
}
ta.tChan.Init()
ta.wg.Add(1) ta.wg.Add(1)
go ta.mainLoop() go ta.mainLoop()
return nil return nil
} }
func (ta *Allocator) connectMaster() error { func (ta *Allocator) Init() {
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second) ta.ForceSyncChan = make(chan Request, maxConcurrentRequests)
defer cancel() ta.Reqs = make(chan Request, maxConcurrentRequests)
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.NewMasterServiceClient(conn)
return nil
}
func (ta *Allocator) init() {
ta.forceSyncChan = make(chan request, maxConcurrentRequests)
} }
func (ta *Allocator) mainLoop() { func (ta *Allocator) mainLoop() {
defer ta.wg.Done() defer ta.wg.Done()
loopCtx, loopCancel := context.WithCancel(ta.ctx) loopCtx, loopCancel := context.WithCancel(ta.Ctx)
defer loopCancel() defer loopCancel()
for { for {
select { select {
case first := <-ta.forceSyncChan: case first := <-ta.ForceSyncChan:
ta.syncReqs = append(ta.syncReqs, first) ta.SyncReqs = append(ta.SyncReqs, first)
pending := len(ta.forceSyncChan) pending := len(ta.ForceSyncChan)
for i := 0; i < pending; i++ { for i := 0; i < pending; i++ {
ta.syncReqs = append(ta.syncReqs, <-ta.forceSyncChan) ta.SyncReqs = append(ta.SyncReqs, <-ta.ForceSyncChan)
} }
ta.sync(true) ta.sync(true)
ta.finishSyncRequest() ta.finishSyncRequest()
case <-ta.tChan.Chan(): case <-ta.TChan.Chan():
ta.pickCanDo() ta.pickCanDo()
ta.finishRequest() ta.finishRequest()
if ta.sync(true) { if ta.sync(true) {
@ -197,11 +156,11 @@ func (ta *Allocator) mainLoop() {
} }
ta.failRemainRequest() ta.failRemainRequest()
case first := <-ta.reqs: case first := <-ta.Reqs:
ta.toDoReqs = append(ta.toDoReqs, first) ta.ToDoReqs = append(ta.ToDoReqs, first)
pending := len(ta.reqs) pending := len(ta.Reqs)
for i := 0; i < pending; i++ { for i := 0; i < pending; i++ {
ta.toDoReqs = append(ta.toDoReqs, <-ta.reqs) ta.ToDoReqs = append(ta.ToDoReqs, <-ta.Reqs)
} }
ta.pickCanDo() ta.pickCanDo()
ta.finishRequest() ta.finishRequest()
@ -219,78 +178,78 @@ func (ta *Allocator) mainLoop() {
} }
func (ta *Allocator) pickCanDo() { func (ta *Allocator) pickCanDo() {
if ta.pickCanDoFunc == nil { if ta.PickCanDoFunc == nil {
return return
} }
ta.pickCanDoFunc() ta.PickCanDoFunc()
} }
func (ta *Allocator) sync(timeout bool) bool { func (ta *Allocator) sync(timeout bool) bool {
if ta.syncFunc == nil || ta.checkSyncFunc == nil { if ta.SyncFunc == nil || ta.CheckSyncFunc == nil {
ta.canDoReqs = ta.toDoReqs ta.CanDoReqs = ta.ToDoReqs
ta.toDoReqs = ta.toDoReqs[0:0] ta.ToDoReqs = ta.ToDoReqs[0:0]
return true return true
} }
if !timeout && len(ta.toDoReqs) == 0 { if !timeout && len(ta.ToDoReqs) == 0 {
return false return false
} }
if !ta.checkSyncFunc(timeout) { if !ta.CheckSyncFunc(timeout) {
return false return false
} }
ret := ta.syncFunc() ret := ta.SyncFunc()
if !timeout { if !timeout {
ta.tChan.Reset() ta.TChan.Reset()
} }
return ret return ret
} }
func (ta *Allocator) finishSyncRequest() { func (ta *Allocator) finishSyncRequest() {
for _, req := range ta.syncReqs { for _, req := range ta.SyncReqs {
if req != nil { if req != nil {
req.Notify(nil) req.Notify(nil)
} }
} }
ta.syncReqs = ta.syncReqs[0:0] ta.SyncReqs = ta.SyncReqs[0:0]
} }
func (ta *Allocator) failRemainRequest() { func (ta *Allocator) failRemainRequest() {
for _, req := range ta.toDoReqs { for _, req := range ta.ToDoReqs {
if req != nil { if req != nil {
req.Notify(errors.New("failed: unexpected error")) req.Notify(errors.New("failed: unexpected error"))
} }
} }
ta.toDoReqs = []request{} ta.ToDoReqs = []Request{}
} }
func (ta *Allocator) finishRequest() { func (ta *Allocator) finishRequest() {
for _, req := range ta.canDoReqs { for _, req := range ta.CanDoReqs {
if req != nil { if req != nil {
err := ta.processFunc(req) err := ta.ProcessFunc(req)
req.Notify(err) req.Notify(err)
} }
} }
ta.canDoReqs = []request{} ta.CanDoReqs = []Request{}
} }
func (ta *Allocator) revokeRequest(err error) { func (ta *Allocator) revokeRequest(err error) {
n := len(ta.reqs) n := len(ta.Reqs)
for i := 0; i < n; i++ { for i := 0; i < n; i++ {
req := <-ta.reqs req := <-ta.Reqs
req.Notify(err) req.Notify(err)
} }
} }
func (ta *Allocator) Close() { func (ta *Allocator) Close() {
ta.cancel() ta.CancelFunc()
ta.wg.Wait() ta.wg.Wait()
ta.tChan.Close() ta.TChan.Close()
ta.revokeRequest(errors.New("closing")) ta.revokeRequest(errors.New("closing"))
} }
func (ta *Allocator) CleanCache() { func (ta *Allocator) CleanCache() {
req := &syncRequest{baseRequest: baseRequest{done: make(chan error), valid: false}} req := &SyncRequest{BaseRequest: BaseRequest{Done: make(chan error), Valid: false}}
ta.forceSyncChan <- req ta.ForceSyncChan <- req
req.Wait() req.Wait()
} }

View File

@ -5,6 +5,9 @@ import (
"log" "log"
"time" "time"
"github.com/zilliztech/milvus-distributed/internal/util/retry"
"google.golang.org/grpc"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb" "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb" "github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil" "github.com/zilliztech/milvus-distributed/internal/util/typeutil"
@ -19,6 +22,12 @@ type UniqueID = typeutil.UniqueID
type IDAllocator struct { type IDAllocator struct {
Allocator Allocator
masterAddress string
masterConn *grpc.ClientConn
masterClient masterpb.MasterServiceClient
countPerRPC uint32
idStart UniqueID idStart UniqueID
idEnd UniqueID idEnd UniqueID
@ -29,22 +38,48 @@ func NewIDAllocator(ctx context.Context, masterAddr string) (*IDAllocator, error
ctx1, cancel := context.WithCancel(ctx) ctx1, cancel := context.WithCancel(ctx)
a := &IDAllocator{ a := &IDAllocator{
Allocator: Allocator{reqs: make(chan request, maxConcurrentRequests), Allocator: Allocator{
ctx: ctx1, Ctx: ctx1,
cancel: cancel, CancelFunc: cancel,
masterAddress: masterAddr,
countPerRPC: IDCountPerRPC,
}, },
countPerRPC: IDCountPerRPC,
masterAddress: masterAddr,
} }
a.tChan = &emptyTicker{} a.TChan = &EmptyTicker{}
a.Allocator.syncFunc = a.syncID a.Allocator.SyncFunc = a.syncID
a.Allocator.processFunc = a.processFunc a.Allocator.ProcessFunc = a.processFunc
a.Allocator.checkSyncFunc = a.checkSyncFunc a.Allocator.CheckSyncFunc = a.checkSyncFunc
a.Allocator.pickCanDoFunc = a.pickCanDoFunc a.Allocator.PickCanDoFunc = a.pickCanDoFunc
a.init() a.Init()
return a, nil return a, nil
} }
func (ia *IDAllocator) Start() error {
connectMasterFn := func() error {
return ia.connectMaster()
}
err := retry.Retry(10, time.Millisecond*200, connectMasterFn)
if err != nil {
panic("connect to master failed")
}
ia.Allocator.Start()
return nil
}
func (ia *IDAllocator) connectMaster() error {
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
defer cancel()
conn, err := grpc.DialContext(ctx, ia.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", ia.masterAddress)
ia.masterConn = conn
ia.masterClient = masterpb.NewMasterServiceClient(conn)
return nil
}
func (ia *IDAllocator) syncID() bool { func (ia *IDAllocator) syncID() bool {
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
req := &masterpb.IDRequest{ req := &masterpb.IDRequest{
@ -69,28 +104,28 @@ func (ia *IDAllocator) syncID() bool {
} }
func (ia *IDAllocator) checkSyncFunc(timeout bool) bool { func (ia *IDAllocator) checkSyncFunc(timeout bool) bool {
return timeout || len(ia.toDoReqs) > 0 return timeout || len(ia.ToDoReqs) > 0
} }
func (ia *IDAllocator) pickCanDoFunc() { func (ia *IDAllocator) pickCanDoFunc() {
total := uint32(ia.idEnd - ia.idStart) total := uint32(ia.idEnd - ia.idStart)
need := uint32(0) need := uint32(0)
idx := 0 idx := 0
for _, req := range ia.toDoReqs { for _, req := range ia.ToDoReqs {
iReq := req.(*idRequest) iReq := req.(*IDRequest)
need += iReq.count need += iReq.count
if need <= total { if need <= total {
ia.canDoReqs = append(ia.canDoReqs, req) ia.CanDoReqs = append(ia.CanDoReqs, req)
idx++ idx++
} else { } else {
break break
} }
} }
ia.toDoReqs = ia.toDoReqs[idx:] ia.ToDoReqs = ia.ToDoReqs[idx:]
} }
func (ia *IDAllocator) processFunc(req request) error { func (ia *IDAllocator) processFunc(req Request) error {
idRequest := req.(*idRequest) idRequest := req.(*IDRequest)
idRequest.id = ia.idStart idRequest.id = ia.idStart
ia.idStart++ ia.idStart++
return nil return nil
@ -105,10 +140,10 @@ func (ia *IDAllocator) AllocOne() (UniqueID, error) {
} }
func (ia *IDAllocator) Alloc(count uint32) (UniqueID, UniqueID, error) { func (ia *IDAllocator) Alloc(count uint32) (UniqueID, UniqueID, error) {
req := &idRequest{baseRequest: baseRequest{done: make(chan error), valid: false}} req := &IDRequest{BaseRequest: BaseRequest{Done: make(chan error), Valid: false}}
req.count = count req.count = count
ia.reqs <- req ia.Reqs <- req
req.Wait() req.Wait()
if !req.IsValid() { if !req.IsValid() {

View File

@ -1,40 +0,0 @@
package allocator
import (
"log"
"time"
)
// Reference: https://blog.cyeam.com/golang/2018/08/27/retry
func RetryImpl(attempts int, sleep time.Duration, fn func() error, maxSleepTime time.Duration) error {
if err := fn(); err != nil {
if s, ok := err.(InterruptError); ok {
return s.error
}
if attempts--; attempts > 0 {
log.Printf("retry func error: %s. attempts #%d after %s.", err.Error(), attempts, sleep)
time.Sleep(sleep)
if sleep < maxSleepTime {
return RetryImpl(attempts, 2*sleep, fn, maxSleepTime)
}
return RetryImpl(attempts, maxSleepTime, fn, maxSleepTime)
}
return err
}
return nil
}
func Retry(attempts int, sleep time.Duration, fn func() error) error {
maxSleepTime := time.Millisecond * 1000
return RetryImpl(attempts, sleep, fn, maxSleepTime)
}
type InterruptError struct {
error
}
func NoRetryError(err error) InterruptError {
return InterruptError{err}
}

View File

@ -3,6 +3,9 @@ package allocator
import ( import (
"context" "context"
"github.com/zilliztech/milvus-distributed/internal/util/retry"
"google.golang.org/grpc"
"log" "log"
"time" "time"
@ -19,6 +22,12 @@ const (
type TimestampAllocator struct { type TimestampAllocator struct {
Allocator Allocator
masterAddress string
masterConn *grpc.ClientConn
masterClient masterpb.MasterServiceClient
countPerRPC uint32
lastTsBegin Timestamp lastTsBegin Timestamp
lastTsEnd Timestamp lastTsEnd Timestamp
PeerID UniqueID PeerID UniqueID
@ -27,42 +36,69 @@ type TimestampAllocator struct {
func NewTimestampAllocator(ctx context.Context, masterAddr string) (*TimestampAllocator, error) { func NewTimestampAllocator(ctx context.Context, masterAddr string) (*TimestampAllocator, error) {
ctx1, cancel := context.WithCancel(ctx) ctx1, cancel := context.WithCancel(ctx)
a := &TimestampAllocator{ a := &TimestampAllocator{
Allocator: Allocator{reqs: make(chan request, maxConcurrentRequests), Allocator: Allocator{
ctx: ctx1, Ctx: ctx1,
cancel: cancel, CancelFunc: cancel,
masterAddress: masterAddr,
countPerRPC: tsCountPerRPC,
}, },
masterAddress: masterAddr,
countPerRPC: tsCountPerRPC,
} }
a.tChan = &ticker{ a.TChan = &Ticker{
updateInterval: time.Second, UpdateInterval: time.Second,
} }
a.Allocator.syncFunc = a.syncTs a.Allocator.SyncFunc = a.syncTs
a.Allocator.processFunc = a.processFunc a.Allocator.ProcessFunc = a.processFunc
a.Allocator.checkSyncFunc = a.checkSyncFunc a.Allocator.CheckSyncFunc = a.checkSyncFunc
a.Allocator.pickCanDoFunc = a.pickCanDoFunc a.Allocator.PickCanDoFunc = a.pickCanDoFunc
a.Init()
return a, nil return a, nil
} }
func (ta *TimestampAllocator) Start() error {
connectMasterFn := func() error {
return ta.connectMaster()
}
err := retry.Retry(10, time.Millisecond*200, connectMasterFn)
if err != nil {
panic("Timestamp local allocator connect to master failed")
}
ta.Allocator.Start()
return nil
}
func (ta *TimestampAllocator) connectMaster() error {
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
defer cancel()
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.NewMasterServiceClient(conn)
return nil
}
func (ta *TimestampAllocator) checkSyncFunc(timeout bool) bool { func (ta *TimestampAllocator) checkSyncFunc(timeout bool) bool {
return timeout || len(ta.toDoReqs) > 0 return timeout || len(ta.ToDoReqs) > 0
} }
func (ta *TimestampAllocator) pickCanDoFunc() { func (ta *TimestampAllocator) pickCanDoFunc() {
total := uint32(ta.lastTsEnd - ta.lastTsBegin) total := uint32(ta.lastTsEnd - ta.lastTsBegin)
need := uint32(0) need := uint32(0)
idx := 0 idx := 0
for _, req := range ta.toDoReqs { for _, req := range ta.ToDoReqs {
tReq := req.(*tsoRequest) tReq := req.(*TSORequest)
need += tReq.count need += tReq.count
if need <= total { if need <= total {
ta.canDoReqs = append(ta.canDoReqs, req) ta.CanDoReqs = append(ta.CanDoReqs, req)
idx++ idx++
} else { } else {
break break
} }
} }
ta.toDoReqs = ta.toDoReqs[idx:] ta.ToDoReqs = ta.ToDoReqs[idx:]
} }
func (ta *TimestampAllocator) syncTs() bool { func (ta *TimestampAllocator) syncTs() bool {
@ -88,8 +124,8 @@ func (ta *TimestampAllocator) syncTs() bool {
return true return true
} }
func (ta *TimestampAllocator) processFunc(req request) error { func (ta *TimestampAllocator) processFunc(req Request) error {
tsoRequest := req.(*tsoRequest) tsoRequest := req.(*TSORequest)
tsoRequest.timestamp = ta.lastTsBegin tsoRequest.timestamp = ta.lastTsBegin
ta.lastTsBegin++ ta.lastTsBegin++
return nil return nil
@ -104,11 +140,11 @@ func (ta *TimestampAllocator) AllocOne() (Timestamp, error) {
} }
func (ta *TimestampAllocator) Alloc(count uint32) ([]Timestamp, error) { func (ta *TimestampAllocator) Alloc(count uint32) ([]Timestamp, error) {
req := &tsoRequest{ req := &TSORequest{
baseRequest: baseRequest{done: make(chan error), valid: false}, BaseRequest: BaseRequest{Done: make(chan error), Valid: false},
} }
req.count = count req.count = count
ta.reqs <- req ta.Reqs <- req
req.Wait() req.Wait()
if !req.IsValid() { if !req.IsValid() {

View File

@ -49,6 +49,7 @@ func (c *GrpcClient) Init() error {
c.grpcClient = masterpb.NewMasterServiceClient(c.conn) c.grpcClient = masterpb.NewMasterServiceClient(c.conn)
return nil return nil
} }
func (c *GrpcClient) Start() error { func (c *GrpcClient) Start() error {
return nil return nil
} }

View File

@ -9,7 +9,6 @@ import (
"github.com/zilliztech/milvus-distributed/internal/errors" "github.com/zilliztech/milvus-distributed/internal/errors"
cms "github.com/zilliztech/milvus-distributed/internal/masterservice" cms "github.com/zilliztech/milvus-distributed/internal/masterservice"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb" "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2" "github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb" "github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb" "github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
@ -227,13 +226,3 @@ func (s *GrpcServer) DescribeSegment(ctx context.Context, in *milvuspb.DescribeS
func (s *GrpcServer) ShowSegments(ctx context.Context, in *milvuspb.ShowSegmentRequest) (*milvuspb.ShowSegmentResponse, error) { func (s *GrpcServer) ShowSegments(ctx context.Context, in *milvuspb.ShowSegmentRequest) (*milvuspb.ShowSegmentResponse, error) {
return s.core.ShowSegments(in) return s.core.ShowSegments(in)
} }
//TODO, move to query node
func (s *GrpcServer) GetIndexState(ctx context.Context, request *milvuspb.IndexStateRequest) (*milvuspb.IndexStateResponse, error) {
panic("implement me")
}
//TODO, move to data service
func (s *GrpcServer) AssignSegmentID(ctx context.Context, request *datapb.AssignSegIDRequest) (*datapb.AssignSegIDResponse, error) {
panic("implement me")
}

View File

@ -2,37 +2,45 @@ package grpcproxynode
import ( import (
"context" "context"
"time"
"google.golang.org/grpc"
"github.com/zilliztech/milvus-distributed/internal/proto/proxypb" "github.com/zilliztech/milvus-distributed/internal/proto/proxypb"
"github.com/zilliztech/milvus-distributed/internal/util/retry"
"google.golang.org/grpc"
) )
type Client struct { type Client struct {
client proxypb.ProxyNodeServiceClient grpcClient proxypb.ProxyNodeServiceClient
address string address string
ctx context.Context ctx context.Context
} }
func (c *Client) tryConnect() error { func (c *Client) Init() error {
if c.client != nil { connectGrpcFunc := func() error {
conn, err := grpc.DialContext(c.ctx, c.address, grpc.WithInsecure(), grpc.WithBlock())
if err != nil {
return err
}
c.grpcClient = proxypb.NewProxyServiceClient(conn)
return nil return nil
} }
conn, err := grpc.DialContext(c.ctx, c.address, grpc.WithInsecure(), grpc.WithBlock()) err := retry.Retry(10, time.Millisecond*200, connectGrpcFunc)
if err != nil { if err != nil {
return err return err
} }
c.client = proxypb.NewProxyNodeServiceClient(conn) return nil
}
func (c *Client) Start() error {
return nil
}
func (c *Client) Stop() error {
return nil return nil
} }
func (c *Client) InvalidateCollectionMetaCache(request *proxypb.InvalidateCollMetaCacheRequest) error { func (c *Client) InvalidateCollectionMetaCache(request *proxypb.InvalidateCollMetaCacheRequest) error {
var err error _, err := c.grpcClient.InvalidateCollectionMetaCache(c.ctx, request)
err = c.tryConnect()
if err != nil {
return err
}
_, err = c.client.InvalidateCollectionMetaCache(c.ctx, request)
return err return err
} }

View File

@ -1,24 +0,0 @@
package grpcproxynode
const (
StartParamsKey = "START_PARAMS"
MasterPort = "master.port"
MasterHost = "master.address"
PulsarPort = "pulsar.port"
PulsarHost = "pulsar.address"
IndexServerPort = "indexBuilder.port"
IndexServerHost = "indexBuilder.address"
QueryNodeIDList = "nodeID.queryNodeIDList"
TimeTickInterval = "proxyNode.timeTickInterval"
SubName = "msgChannel.subNamePrefix.proxySubNamePrefix"
TimeTickChannelNames = "msgChannel.chanNamePrefix.proxyTimeTick"
MsgStreamInsertBufSize = "proxyNode.msgStream.insert.bufSize"
MsgStreamSearchBufSize = "proxyNode.msgStream.search.bufSize"
MsgStreamSearchResultBufSize = "proxyNode.msgStream.searchResult.recvBufSize"
MsgStreamSearchResultPulsarBufSize = "proxyNode.msgStream.searchResult.pulsarBufSize"
MsgStreamTimeTickBufSize = "proxyNode.msgStream.timeTick.bufSize"
MaxNameLength = "proxyNode.maxNameLength"
MaxFieldNum = "proxyNode.maxFieldNum"
MaxDimension = "proxyNode.MaxDimension"
DefaultPartitionTag = "common.defaultPartitionTag"
)

View File

@ -2,6 +2,7 @@ package grpcproxynode
import ( import (
"net" "net"
"os"
"strconv" "strconv"
"github.com/zilliztech/milvus-distributed/internal/util/paramtable" "github.com/zilliztech/milvus-distributed/internal/util/paramtable"
@ -11,14 +12,68 @@ type ParamTable struct {
paramtable.BaseTable paramtable.BaseTable
ProxyServiceAddress string ProxyServiceAddress string
ProxyServicePort int
IndexServerAddress string
MasterAddress string
DataServiceAddress string
QueryServiceAddress string
IP string
Port int
Address string
} }
var Params ParamTable var Params ParamTable
func (pt *ParamTable) Init() { func (pt *ParamTable) Init() {
pt.BaseTable.Init() pt.BaseTable.Init()
pt.initParams()
}
// todo
func (pt *ParamTable) LoadFromArgs() {
}
//todo
func (pt *ParamTable) LoadFromEnv() {
masterAddress := os.Getenv("MASTER_ADDRESS")
if masterAddress != "" {
pt.MasterAddress = masterAddress
}
indexServiceAddress := os.Getenv("INDEX_SERVICE_ADDRESS")
if indexServiceAddress != "" {
pt.IndexServerAddress = indexServiceAddress
}
queryServiceAddress := os.Getenv("QUERY_SERVICE_ADDRESS")
if queryServiceAddress != "" {
pt.QueryServiceAddress = queryServiceAddress
}
dataServiceAddress := os.Getenv("DATA_SERVICE_ADDRESS")
if dataServiceAddress != "" {
pt.DataServiceAddress = dataServiceAddress
}
}
func (pt *ParamTable) initParams() {
pt.initPoxyServicePort()
pt.initProxyServiceAddress() pt.initProxyServiceAddress()
pt.initMasterAddress()
pt.initIndexServerAddress()
pt.initDataServiceAddress()
pt.initQueryServiceAddress()
}
func (pt *ParamTable) initPoxyServicePort() {
pt.ProxyServicePort = pt.ParseInt("proxyService.port")
} }
func (pt *ParamTable) initProxyServiceAddress() { func (pt *ParamTable) initProxyServiceAddress() {
@ -44,3 +99,72 @@ func (pt *ParamTable) initProxyServiceAddress() {
} }
pt.ProxyServiceAddress = addr + ":" + port pt.ProxyServiceAddress = addr + ":" + port
} }
// todo remove and use load from env
func (pt *ParamTable) initIndexServerAddress() {
addr, err := pt.Load("indexServer.address")
if err != nil {
panic(err)
}
hostName, _ := net.LookupHost(addr)
if len(hostName) <= 0 {
if ip := net.ParseIP(addr); ip == nil {
panic("invalid ip indexServer.address")
}
}
port, err := pt.Load("indexServer.port")
if err != nil {
panic(err)
}
_, err = strconv.Atoi(port)
if err != nil {
panic(err)
}
pt.IndexServerAddress = addr + ":" + port
}
// todo remove and use load from env
func (pt *ParamTable) initMasterAddress() {
masterHost, err := pt.Load("master.address")
if err != nil {
panic(err)
}
port, err := pt.Load("master.port")
if err != nil {
panic(err)
}
pt.MasterAddress = masterHost + ":" + port
}
// todo remove and use load from env
func (pt *ParamTable) initDataServiceAddress() {
addr, err := pt.Load("dataService.address")
if err != nil {
panic(err)
}
port, err := pt.Load("dataService.port")
if err != nil {
panic(err)
}
pt.DataServiceAddress = addr + ":" + port
}
// todo remove and use load from env
func (pt *ParamTable) initQueryServiceAddress() {
addr, err := pt.Load("queryService.address")
if err != nil {
panic(err)
}
port, err := pt.Load("queryService.port")
if err != nil {
panic(err)
}
pt.QueryServiceAddress = addr + ":" + port
}

View File

@ -1,239 +1,196 @@
package grpcproxynode package grpcproxynode
import ( import (
"bytes"
"context" "context"
"log"
"net" "net"
"os" "os"
"strconv" "strconv"
"strings"
"sync" "sync"
"time" "time"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil" "github.com/zilliztech/milvus-distributed/internal/util/funcutil"
"github.com/spf13/viper"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2" "github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
"github.com/go-basic/ipv4" "google.golang.org/grpc"
grpcdataservice "github.com/zilliztech/milvus-distributed/internal/distributed/dataservice"
grpcindexserviceclient "github.com/zilliztech/milvus-distributed/internal/distributed/indexservice/client"
grcpmasterservice "github.com/zilliztech/milvus-distributed/internal/distributed/masterservice"
grpcproxyservice "github.com/zilliztech/milvus-distributed/internal/distributed/proxyservice" grpcproxyservice "github.com/zilliztech/milvus-distributed/internal/distributed/proxyservice"
grpcqueryserviceclient "github.com/zilliztech/milvus-distributed/internal/distributed/queryservice/client"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb" "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb" "github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
"github.com/zilliztech/milvus-distributed/internal/proxynode"
"github.com/zilliztech/milvus-distributed/internal/proto/proxypb" "github.com/zilliztech/milvus-distributed/internal/proto/proxypb"
"github.com/zilliztech/milvus-distributed/internal/proxynode"
"google.golang.org/grpc"
) )
type Server struct { type Server struct {
ctx context.Context ctx context.Context
wg sync.WaitGroup wg sync.WaitGroup
impl proxynode.ProxyNode impl *proxynode.NodeImpl
grpcServer *grpc.Server grpcServer *grpc.Server
ip string
port int grpcErrChan chan error
proxyServiceAddress string
proxyServiceClient *grpcproxyservice.Client ip string
port int
//todo
proxyServiceClient *grpcproxyservice.Client
// todo InitParams Service addrs
masterServiceClient *grcpmasterservice.GrpcClient
dataServiceClient *grpcdataservice.Client
queryServiceClient *grpcqueryserviceclient.Client
indexServiceClient *grpcindexserviceclient.Client
} }
func CreateProxyNodeServer() (*Server, error) { func NewServer() (*Server, error) {
return &Server{}, nil
}
func (s *Server) loadConfigFromInitParams(initParams *internalpb2.InitParams) error { server := &Server{
proxynode.Params.ProxyID = initParams.NodeID ctx: context.Background(),
grpcErrChan: make(chan error),
config := viper.New()
config.SetConfigType("yaml")
for _, pair := range initParams.StartParams {
if pair.Key == StartParamsKey {
err := config.ReadConfig(bytes.NewBuffer([]byte(pair.Value)))
if err != nil {
return err
}
break
}
} }
masterPort := config.GetString(MasterPort) var err error
masterHost := config.GetString(MasterHost) server.impl, err = proxynode.NewProxyNodeImpl(server.ctx)
proxynode.Params.MasterAddress = masterHost + ":" + masterPort if err != nil {
return nil, err
}
return server, err
}
pulsarPort := config.GetString(PulsarPort) func (s *Server) startGrpcLoop(grpcPort int) {
pulsarHost := config.GetString(PulsarHost)
proxynode.Params.PulsarAddress = pulsarHost + ":" + pulsarPort
indexServerPort := config.GetString(IndexServerPort) defer s.wg.Done()
indexServerHost := config.GetString(IndexServerHost)
proxynode.Params.IndexServerAddress = indexServerHost + ":" + indexServerPort
queryNodeIDList := config.GetString(QueryNodeIDList) log.Println("network port: ", grpcPort)
proxynode.Params.QueryNodeIDList = nil lis, err := net.Listen("tcp", ":"+strconv.Itoa(grpcPort))
queryNodeIDs := strings.Split(queryNodeIDList, ",") if err != nil {
for _, queryNodeID := range queryNodeIDs { log.Printf("GrpcServer:failed to listen: %v", err)
v, err := strconv.Atoi(queryNodeID) s.grpcErrChan <- err
return
}
ctx, cancel := context.WithCancel(s.ctx)
defer cancel()
s.grpcServer = grpc.NewServer()
proxypb.RegisterProxyNodeServiceServer(s.grpcServer, s)
milvuspb.RegisterMilvusServiceServer(s.grpcServer, s)
go funcutil.CheckGrpcReady(ctx, s.grpcErrChan)
if err := s.grpcServer.Serve(lis); err != nil {
s.grpcErrChan <- err
}
}
func (s *Server) Run() error {
if err := s.init(); err != nil {
return nil
}
if err := s.start(); err != nil {
return err
}
return nil
}
func (s *Server) init() error {
var err error
Params.Init()
Params.IP = funcutil.GetLocalIP()
host := os.Getenv("PROXY_NODE_HOST")
if len(host) > 0 {
Params.IP = host
}
Params.Port = funcutil.GetAvailablePort()
Params.Address = Params.IP + ":" + strconv.FormatInt(int64(Params.Port), 10)
defer func() {
if err != nil { if err != nil {
return err err2 := s.Stop()
if err2 != nil {
log.Println("Init failed, and Stop failed")
}
} }
proxynode.Params.QueryNodeIDList = append(proxynode.Params.QueryNodeIDList, typeutil.UniqueID(v)) }()
}
proxynode.Params.QueryNodeNum = len(proxynode.Params.QueryNodeIDList)
timeTickInterval := config.GetString(TimeTickInterval) s.wg.Add(1)
interval, err := strconv.Atoi(timeTickInterval) s.startGrpcLoop(Params.Port)
// wait for grpc server loop start
err = <-s.grpcErrChan
if err != nil { if err != nil {
return err return err
} }
proxynode.Params.TimeTickInterval = time.Duration(interval) * time.Millisecond
subName := config.GetString(SubName) s.proxyServiceClient = grpcproxyservice.NewClient(Params.ProxyServiceAddress)
proxynode.Params.ProxySubName = subName err = s.proxyServiceClient.Init()
timeTickChannelNames := config.GetString(TimeTickChannelNames)
proxynode.Params.ProxyTimeTickChannelNames = []string{timeTickChannelNames}
msgStreamInsertBufSizeStr := config.GetString(MsgStreamInsertBufSize)
msgStreamInsertBufSize, err := strconv.Atoi(msgStreamInsertBufSizeStr)
if err != nil { if err != nil {
return err return err
} }
proxynode.Params.MsgStreamInsertBufSize = int64(msgStreamInsertBufSize) s.impl.SetProxyServiceClient(s.proxyServiceClient)
msgStreamSearchBufSizeStr := config.GetString(MsgStreamSearchBufSize) masterServiceAddr := Params.MasterAddress
msgStreamSearchBufSize, err := strconv.Atoi(msgStreamSearchBufSizeStr) timeout := 3 * time.Second
s.masterServiceClient, err = grcpmasterservice.NewGrpcClient(masterServiceAddr, timeout)
if err != nil { if err != nil {
return err return err
} }
proxynode.Params.MsgStreamSearchBufSize = int64(msgStreamSearchBufSize) err = s.masterServiceClient.Init()
msgStreamSearchResultBufSizeStr := config.GetString(MsgStreamSearchResultBufSize)
msgStreamSearchResultBufSize, err := strconv.Atoi(msgStreamSearchResultBufSizeStr)
if err != nil { if err != nil {
return err return err
} }
proxynode.Params.MsgStreamSearchResultBufSize = int64(msgStreamSearchResultBufSize) s.impl.SetMasterClient(s.masterServiceClient)
msgStreamSearchResultPulsarBufSizeStr := config.GetString(MsgStreamSearchResultPulsarBufSize) dataServiceAddr := Params.DataServiceAddress
msgStreamSearchResultPulsarBufSize, err := strconv.Atoi(msgStreamSearchResultPulsarBufSizeStr) s.dataServiceClient = grpcdataservice.NewClient(dataServiceAddr)
err = s.dataServiceClient.Init()
if err != nil { if err != nil {
return err return err
} }
proxynode.Params.MsgStreamSearchResultPulsarBufSize = int64(msgStreamSearchResultPulsarBufSize)
msgStreamTimeTickBufSizeStr := config.GetString(MsgStreamTimeTickBufSize) s.impl.SetDataServiceClient(s.dataServiceClient)
msgStreamTimeTickBufSize, err := strconv.Atoi(msgStreamTimeTickBufSizeStr)
indexServiceAddr := Params.IndexServerAddress
s.indexServiceClient = grpcindexserviceclient.NewClient(indexServiceAddr)
err = s.indexServiceClient.Init()
if err != nil { if err != nil {
return err return err
} }
proxynode.Params.MsgStreamTimeTickBufSize = int64(msgStreamTimeTickBufSize) s.impl.SetIndexServiceClient(s.indexServiceClient)
maxNameLengthStr := config.GetString(MaxNameLength) queryServiceAddr := Params.QueryServiceAddress
maxNameLength, err := strconv.Atoi(maxNameLengthStr) s.queryServiceClient = grpcqueryserviceclient.NewClient(queryServiceAddr)
err = s.queryServiceClient.Init()
if err != nil { if err != nil {
return err return err
} }
proxynode.Params.MaxNameLength = int64(maxNameLength) s.impl.SetQueryServiceClient(s.queryServiceClient)
maxFieldNumStr := config.GetString(MaxFieldNum) proxynode.Params.Init()
maxFieldNum, err := strconv.Atoi(maxFieldNumStr) proxynode.Params.NetworkPort = Params.Port
if err != nil { proxynode.Params.IP = Params.IP
proxynode.Params.NetworkAddress = Params.Address
// for purpose of ID Allocator
proxynode.Params.MasterAddress = Params.MasterAddress
s.impl.UpdateStateCode(internalpb2.StateCode_INITIALIZING)
if err := s.impl.Init(); err != nil {
return err return err
} }
proxynode.Params.MaxFieldNum = int64(maxFieldNum)
maxDimensionStr := config.GetString(MaxDimension)
maxDimension, err := strconv.Atoi(maxDimensionStr)
if err != nil {
return err
}
proxynode.Params.MaxDimension = int64(maxDimension)
defaultPartitionTag := config.GetString(DefaultPartitionTag)
proxynode.Params.DefaultPartitionTag = defaultPartitionTag
return nil return nil
} }
func (s *Server) connectProxyService() error { func (s *Server) start() error {
Params.Init()
proxynode.Params.Init()
s.proxyServiceAddress = Params.ProxyServiceAddress
s.proxyServiceClient = grpcproxyservice.NewClient(s.proxyServiceAddress)
getAvailablePort := func() int {
listener, err := net.Listen("tcp", ":0")
if err != nil {
panic(err)
}
defer listener.Close()
return listener.Addr().(*net.TCPAddr).Port
}
getLocalIP := func() string {
localIP := ipv4.LocalIP()
host := os.Getenv("PROXY_NODE_HOST")
// TODO: shall we write this to ParamTable?
if len(host) <= 0 {
return localIP
}
return host
}
s.ip = getLocalIP()
s.port = getAvailablePort()
request := &proxypb.RegisterNodeRequest{
Address: &commonpb.Address{
Ip: s.ip,
Port: int64(s.port),
},
}
response, err := s.proxyServiceClient.RegisterNode(request)
if err != nil {
panic(err)
}
return s.loadConfigFromInitParams(response.InitParams)
}
func (s *Server) Init() error {
s.ctx = context.Background()
var err error
s.impl, err = proxynode.CreateProxyNodeImpl(s.ctx)
if err != nil {
return err
}
err = s.connectProxyService()
if err != nil {
return err
}
return s.impl.Init()
}
func (s *Server) Start() error {
s.wg.Add(1)
go func() {
defer s.wg.Done()
// TODO: use config
lis, err := net.Listen("tcp", ":"+strconv.Itoa(s.port))
if err != nil {
panic(err)
}
s.grpcServer = grpc.NewServer()
proxypb.RegisterProxyNodeServiceServer(s.grpcServer, s)
milvuspb.RegisterMilvusServiceServer(s.grpcServer, s)
if err = s.grpcServer.Serve(lis); err != nil {
panic(err)
}
}()
return s.impl.Start() return s.impl.Start()
} }
@ -259,89 +216,89 @@ func (s *Server) InvalidateCollectionMetaCache(ctx context.Context, request *pro
} }
func (s *Server) CreateCollection(ctx context.Context, request *milvuspb.CreateCollectionRequest) (*commonpb.Status, error) { func (s *Server) CreateCollection(ctx context.Context, request *milvuspb.CreateCollectionRequest) (*commonpb.Status, error) {
return s.impl.CreateCollection(ctx, request) return s.impl.CreateCollection(request)
} }
func (s *Server) DropCollection(ctx context.Context, request *milvuspb.DropCollectionRequest) (*commonpb.Status, error) { func (s *Server) DropCollection(ctx context.Context, request *milvuspb.DropCollectionRequest) (*commonpb.Status, error) {
return s.impl.DropCollection(ctx, request) return s.impl.DropCollection(request)
} }
func (s *Server) HasCollection(ctx context.Context, request *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error) { func (s *Server) HasCollection(ctx context.Context, request *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error) {
return s.impl.HasCollection(ctx, request) return s.impl.HasCollection(request)
} }
func (s *Server) LoadCollection(ctx context.Context, request *milvuspb.LoadCollectionRequest) (*commonpb.Status, error) { func (s *Server) LoadCollection(ctx context.Context, request *milvuspb.LoadCollectionRequest) (*commonpb.Status, error) {
return s.impl.LoadCollection(ctx, request) return s.impl.LoadCollection(request)
} }
func (s *Server) ReleaseCollection(ctx context.Context, request *milvuspb.ReleaseCollectionRequest) (*commonpb.Status, error) { func (s *Server) ReleaseCollection(ctx context.Context, request *milvuspb.ReleaseCollectionRequest) (*commonpb.Status, error) {
return s.impl.ReleaseCollection(ctx, request) return s.impl.ReleaseCollection(request)
} }
func (s *Server) DescribeCollection(ctx context.Context, request *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) { func (s *Server) DescribeCollection(ctx context.Context, request *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
return s.impl.DescribeCollection(ctx, request) return s.impl.DescribeCollection(request)
} }
func (s *Server) GetCollectionStatistics(ctx context.Context, request *milvuspb.CollectionStatsRequest) (*milvuspb.CollectionStatsResponse, error) { func (s *Server) GetCollectionStatistics(ctx context.Context, request *milvuspb.CollectionStatsRequest) (*milvuspb.CollectionStatsResponse, error) {
return s.impl.GetCollectionStatistics(ctx, request) return s.impl.GetCollectionStatistics(request)
} }
func (s *Server) ShowCollections(ctx context.Context, request *milvuspb.ShowCollectionRequest) (*milvuspb.ShowCollectionResponse, error) { func (s *Server) ShowCollections(ctx context.Context, request *milvuspb.ShowCollectionRequest) (*milvuspb.ShowCollectionResponse, error) {
return s.impl.ShowCollections(ctx, request) return s.impl.ShowCollections(request)
} }
func (s *Server) CreatePartition(ctx context.Context, request *milvuspb.CreatePartitionRequest) (*commonpb.Status, error) { func (s *Server) CreatePartition(ctx context.Context, request *milvuspb.CreatePartitionRequest) (*commonpb.Status, error) {
return s.impl.CreatePartition(ctx, request) return s.impl.CreatePartition(request)
} }
func (s *Server) DropPartition(ctx context.Context, request *milvuspb.DropPartitionRequest) (*commonpb.Status, error) { func (s *Server) DropPartition(ctx context.Context, request *milvuspb.DropPartitionRequest) (*commonpb.Status, error) {
return s.impl.DropPartition(ctx, request) return s.impl.DropPartition(request)
} }
func (s *Server) HasPartition(ctx context.Context, request *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error) { func (s *Server) HasPartition(ctx context.Context, request *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error) {
return s.impl.HasPartition(ctx, request) return s.impl.HasPartition(request)
} }
func (s *Server) LoadPartitions(ctx context.Context, request *milvuspb.LoadPartitonRequest) (*commonpb.Status, error) { func (s *Server) LoadPartitions(ctx context.Context, request *milvuspb.LoadPartitonRequest) (*commonpb.Status, error) {
return s.impl.LoadPartitions(ctx, request) return s.impl.LoadPartitions(request)
} }
func (s *Server) ReleasePartitions(ctx context.Context, request *milvuspb.ReleasePartitionRequest) (*commonpb.Status, error) { func (s *Server) ReleasePartitions(ctx context.Context, request *milvuspb.ReleasePartitionRequest) (*commonpb.Status, error) {
return s.impl.ReleasePartitions(ctx, request) return s.impl.ReleasePartitions(request)
} }
func (s *Server) GetPartitionStatistics(ctx context.Context, request *milvuspb.PartitionStatsRequest) (*milvuspb.PartitionStatsResponse, error) { func (s *Server) GetPartitionStatistics(ctx context.Context, request *milvuspb.PartitionStatsRequest) (*milvuspb.PartitionStatsResponse, error) {
return s.impl.GetPartitionStatistics(ctx, request) return s.impl.GetPartitionStatistics(request)
} }
func (s *Server) ShowPartitions(ctx context.Context, request *milvuspb.ShowPartitionRequest) (*milvuspb.ShowPartitionResponse, error) { func (s *Server) ShowPartitions(ctx context.Context, request *milvuspb.ShowPartitionRequest) (*milvuspb.ShowPartitionResponse, error) {
return s.impl.ShowPartitions(ctx, request) return s.impl.ShowPartitions(request)
} }
func (s *Server) CreateIndex(ctx context.Context, request *milvuspb.CreateIndexRequest) (*commonpb.Status, error) { func (s *Server) CreateIndex(ctx context.Context, request *milvuspb.CreateIndexRequest) (*commonpb.Status, error) {
return s.impl.CreateIndex(ctx, request) return s.impl.CreateIndex(request)
} }
func (s *Server) DescribeIndex(ctx context.Context, request *milvuspb.DescribeIndexRequest) (*milvuspb.DescribeIndexResponse, error) { func (s *Server) DescribeIndex(ctx context.Context, request *milvuspb.DescribeIndexRequest) (*milvuspb.DescribeIndexResponse, error) {
return s.impl.DescribeIndex(ctx, request) return s.impl.DescribeIndex(request)
} }
func (s *Server) GetIndexState(ctx context.Context, request *milvuspb.IndexStateRequest) (*milvuspb.IndexStateResponse, error) { func (s *Server) GetIndexState(ctx context.Context, request *milvuspb.IndexStateRequest) (*milvuspb.IndexStateResponse, error) {
return s.impl.GetIndexState(ctx, request) return s.impl.GetIndexState(request)
} }
func (s *Server) Insert(ctx context.Context, request *milvuspb.InsertRequest) (*milvuspb.InsertResponse, error) { func (s *Server) Insert(ctx context.Context, request *milvuspb.InsertRequest) (*milvuspb.InsertResponse, error) {
return s.impl.Insert(ctx, request) return s.impl.Insert(request)
} }
func (s *Server) Search(ctx context.Context, request *milvuspb.SearchRequest) (*milvuspb.SearchResults, error) { func (s *Server) Search(ctx context.Context, request *milvuspb.SearchRequest) (*milvuspb.SearchResults, error) {
return s.impl.Search(ctx, request) return s.impl.Search(request)
} }
func (s *Server) Flush(ctx context.Context, request *milvuspb.FlushRequest) (*commonpb.Status, error) { func (s *Server) Flush(ctx context.Context, request *milvuspb.FlushRequest) (*commonpb.Status, error) {
return s.impl.Flush(ctx, request) return s.impl.Flush(request)
} }
func (s *Server) GetDdChannel(ctx context.Context, request *commonpb.Empty) (*milvuspb.StringResponse, error) { func (s *Server) GetDdChannel(ctx context.Context, request *commonpb.Empty) (*milvuspb.StringResponse, error) {
return s.impl.GetDdChannel(ctx, request) return s.impl.GetDdChannel(request)
} }

View File

@ -2,14 +2,14 @@ package grpcproxyservice
import ( import (
"context" "context"
"time"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"google.golang.org/grpc" "google.golang.org/grpc"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
"github.com/zilliztech/milvus-distributed/internal/proto/proxypb" "github.com/zilliztech/milvus-distributed/internal/proto/proxypb"
"github.com/zilliztech/milvus-distributed/internal/util/retry"
) )
type Client struct { type Client struct {
@ -18,41 +18,40 @@ type Client struct {
ctx context.Context ctx context.Context
} }
func (c *Client) tryConnect() error { func (c *Client) Init() error {
if c.proxyServiceClient != nil { connectGrpcFunc := func() error {
conn, err := grpc.DialContext(c.ctx, c.address, grpc.WithInsecure(), grpc.WithBlock())
if err != nil {
return err
}
c.proxyServiceClient = proxypb.NewProxyServiceClient(conn)
return nil return nil
} }
conn, err := grpc.DialContext(c.ctx, c.address, grpc.WithInsecure(), grpc.WithBlock()) err := retry.Retry(10, time.Millisecond*200, connectGrpcFunc)
if err != nil { if err != nil {
return err return err
} }
c.proxyServiceClient = proxypb.NewProxyServiceClient(conn) return nil
}
func (c *Client) Start() error {
return nil
}
func (c *Client) Stop() error {
return nil return nil
} }
func (c *Client) RegisterNode(request *proxypb.RegisterNodeRequest) (*proxypb.RegisterNodeResponse, error) { func (c *Client) RegisterNode(request *proxypb.RegisterNodeRequest) (*proxypb.RegisterNodeResponse, error) {
err := c.tryConnect()
if err != nil {
return nil, err
}
return c.proxyServiceClient.RegisterNode(c.ctx, request) return c.proxyServiceClient.RegisterNode(c.ctx, request)
} }
func (c *Client) InvalidateCollectionMetaCache(request *proxypb.InvalidateCollMetaCacheRequest) error { func (c *Client) InvalidateCollectionMetaCache(request *proxypb.InvalidateCollMetaCacheRequest) error {
var err error _, err := c.proxyServiceClient.InvalidateCollectionMetaCache(c.ctx, request)
err = c.tryConnect()
if err != nil {
return err
}
_, err = c.proxyServiceClient.InvalidateCollectionMetaCache(c.ctx, request)
return err return err
} }
func (c *Client) GetTimeTickChannel() (string, error) { func (c *Client) GetTimeTickChannel() (string, error) {
err := c.tryConnect()
if err != nil {
return "", err
}
response, err := c.proxyServiceClient.GetTimeTickChannel(c.ctx, &commonpb.Empty{}) response, err := c.proxyServiceClient.GetTimeTickChannel(c.ctx, &commonpb.Empty{})
if err != nil { if err != nil {
return "", err return "", err
@ -61,13 +60,13 @@ func (c *Client) GetTimeTickChannel() (string, error) {
} }
func (c *Client) GetComponentStates() (*internalpb2.ComponentStates, error) { func (c *Client) GetComponentStates() (*internalpb2.ComponentStates, error) {
err := c.tryConnect()
if err != nil {
return nil, err
}
return c.proxyServiceClient.GetComponentStates(c.ctx, &commonpb.Empty{}) return c.proxyServiceClient.GetComponentStates(c.ctx, &commonpb.Empty{})
} }
func (c *Client) GetStatisticsChannel() (string, error) {
return "", nil
}
func NewClient(address string) *Client { func NewClient(address string) *Client {
return &Client{ return &Client{
address: address, address: address,

View File

@ -9,19 +9,28 @@ import (
type ParamTable struct { type ParamTable struct {
paramtable.BaseTable paramtable.BaseTable
ServiceAddress string
ServicePort int
} }
var Params ParamTable var Params ParamTable
func (pt *ParamTable) Init() { func (pt *ParamTable) Init() {
pt.BaseTable.Init() pt.BaseTable.Init()
pt.initParams()
} }
func (pt *ParamTable) NetworkPort() int { func (pt *ParamTable) initParams() {
return pt.ParseInt("proxyService.port") pt.initServicePort()
pt.initServiceAddress()
} }
func (pt *ParamTable) NetworkAddress() string { func (pt *ParamTable) initServicePort() {
pt.ServicePort = pt.ParseInt("proxyService.port")
}
func (pt *ParamTable) initServiceAddress() {
addr, err := pt.Load("proxyService.address") addr, err := pt.Load("proxyService.address")
if err != nil { if err != nil {
panic(err) panic(err)
@ -42,5 +51,5 @@ func (pt *ParamTable) NetworkAddress() string {
if err != nil { if err != nil {
panic(err) panic(err)
} }
return addr + ":" + port pt.ServiceAddress = addr + ":" + port
} }

View File

@ -3,85 +3,106 @@ package grpcproxyservice
import ( import (
"context" "context"
"fmt" "fmt"
"log"
"net" "net"
"strconv" "strconv"
"sync" "sync"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb" "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
"github.com/zilliztech/milvus-distributed/internal/proto/proxypb" "github.com/zilliztech/milvus-distributed/internal/proto/proxypb"
"github.com/zilliztech/milvus-distributed/internal/proxyservice" "github.com/zilliztech/milvus-distributed/internal/proxyservice"
"github.com/zilliztech/milvus-distributed/internal/util/funcutil"
"google.golang.org/grpc" "google.golang.org/grpc"
) )
type Server struct { type Server struct {
ctx context.Context ctx context.Context
wg sync.WaitGroup wg sync.WaitGroup
impl proxyservice.ProxyService
grpcServer *grpc.Server grpcServer *grpc.Server
grpcErrChan chan error
impl *proxyservice.ServiceImpl
} }
func (s *Server) GetTimeTickChannel(ctx context.Context, empty *commonpb.Empty) (*milvuspb.StringResponse, error) { func NewServer() (*Server, error) {
channel, err := s.impl.GetTimeTickChannel()
if err != nil { server := &Server{
return &milvuspb.StringResponse{ ctx: context.Background(),
Status: &commonpb.Status{ grpcErrChan: make(chan error),
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: err.Error(),
},
Value: "",
}, nil
} }
return &milvuspb.StringResponse{
Value: channel, var err error
}, nil server.impl, err = proxyservice.NewServiceImpl(server.ctx)
if err != nil {
return nil, err
}
return server, err
} }
func (s *Server) GetComponentStates(ctx context.Context, empty *commonpb.Empty) (*internalpb2.ComponentStates, error) { func (s *Server) Run() error {
return s.impl.GetComponentStates()
}
func CreateProxyServiceServer() (*Server, error) { if err := s.init(); err != nil {
return &Server{}, nil return err
} }
func (s *Server) Init() error { if err := s.start(); err != nil {
s.ctx = context.Background() return err
Params.Init() }
proxyservice.Params.Init()
s.impl, _ = proxyservice.CreateProxyService(s.ctx)
s.impl.Init()
return nil return nil
} }
func (s *Server) Start() error { func (s *Server) init() error {
fmt.Println("proxy service start ...") Params.Init()
proxyservice.Params.Init()
s.wg.Add(1) s.wg.Add(1)
go func() { s.startGrpcLoop(Params.ServicePort)
defer s.wg.Done() // wait for grpc server loop start
if err := <-s.grpcErrChan; err != nil {
return err
}
s.impl.UpdateStateCode(internalpb2.StateCode_INITIALIZING)
// TODO: use config if err := s.impl.Init(); err != nil {
fmt.Println("network port: ", Params.NetworkPort()) return err
lis, err := net.Listen("tcp", ":"+strconv.Itoa(Params.NetworkPort())) }
if err != nil { return nil
panic(err) }
}
s.grpcServer = grpc.NewServer() func (s *Server) startGrpcLoop(grpcPort int) {
proxypb.RegisterProxyServiceServer(s.grpcServer, s)
milvuspb.RegisterProxyServiceServer(s.grpcServer, s)
if err = s.grpcServer.Serve(lis); err != nil {
panic(err)
}
}()
s.impl.Start() defer s.wg.Done()
fmt.Println("network port: ", grpcPort)
lis, err := net.Listen("tcp", ":"+strconv.Itoa(grpcPort))
if err != nil {
log.Printf("GrpcServer:failed to listen: %v", err)
s.grpcErrChan <- err
return
}
ctx, cancel := context.WithCancel(s.ctx)
defer cancel()
s.grpcServer = grpc.NewServer()
proxypb.RegisterProxyServiceServer(s.grpcServer, s)
milvuspb.RegisterProxyServiceServer(s.grpcServer, s)
go funcutil.CheckGrpcReady(ctx, s.grpcErrChan)
if err := s.grpcServer.Serve(lis); err != nil {
s.grpcErrChan <- err
}
}
func (s *Server) start() error {
fmt.Println("proxy ServiceImpl start ...")
if err := s.impl.Start(); err != nil {
return err
}
return nil return nil
} }
@ -105,3 +126,23 @@ func (s *Server) RegisterNode(ctx context.Context, request *proxypb.RegisterNode
func (s *Server) InvalidateCollectionMetaCache(ctx context.Context, request *proxypb.InvalidateCollMetaCacheRequest) (*commonpb.Status, error) { func (s *Server) InvalidateCollectionMetaCache(ctx context.Context, request *proxypb.InvalidateCollMetaCacheRequest) (*commonpb.Status, error) {
return &commonpb.Status{}, s.impl.InvalidateCollectionMetaCache(request) return &commonpb.Status{}, s.impl.InvalidateCollectionMetaCache(request)
} }
func (s *Server) GetTimeTickChannel(ctx context.Context, empty *commonpb.Empty) (*milvuspb.StringResponse, error) {
channel, err := s.impl.GetTimeTickChannel()
if err != nil {
return &milvuspb.StringResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: err.Error(),
},
Value: "",
}, nil
}
return &milvuspb.StringResponse{
Value: channel,
}, nil
}
func (s *Server) GetComponentStates(ctx context.Context, empty *commonpb.Empty) (*internalpb2.ComponentStates, error) {
return s.impl.GetComponentStates()
}

View File

@ -8,15 +8,10 @@ import (
"testing" "testing"
"time" "time"
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb" "github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
"github.com/golang/protobuf/proto" "github.com/golang/protobuf/proto"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
ms "github.com/zilliztech/milvus-distributed/internal/msgstream" ms "github.com/zilliztech/milvus-distributed/internal/msgstream"
@ -25,6 +20,7 @@ import (
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb" "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb" "github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb" "github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
"go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/clientv3"
"go.uber.org/zap" "go.uber.org/zap"
"google.golang.org/grpc" "google.golang.org/grpc"
@ -1110,110 +1106,110 @@ func TestMaster(t *testing.T) {
assert.Equal(t, dropCollectionReq.CollectionName, dropCollectionMsg.DropCollectionRequest.CollectionName) assert.Equal(t, dropCollectionReq.CollectionName, dropCollectionMsg.DropCollectionRequest.CollectionName)
}) })
t.Run("TestSegmentManager_RPC", func(t *testing.T) { //t.Run("TestSegmentManager_RPC", func(t *testing.T) {
collName := "test_coll" // collName := "test_coll"
partitionName := "test_part" // partitionName := "test_part"
schema := &schemapb.CollectionSchema{ // schema := &schemapb.CollectionSchema{
Name: collName, // Name: collName,
Description: "test coll", // Description: "test coll",
AutoID: false, // AutoID: false,
Fields: []*schemapb.FieldSchema{ // Fields: []*schemapb.FieldSchema{
{FieldID: 1, Name: "f1", IsPrimaryKey: false, DataType: schemapb.DataType_INT32}, // {FieldID: 1, Name: "f1", IsPrimaryKey: false, DataType: schemapb.DataType_INT32},
{FieldID: 1, Name: "f1", IsPrimaryKey: false, DataType: schemapb.DataType_VECTOR_FLOAT, TypeParams: []*commonpb.KeyValuePair{{Key: "dim", Value: "128"}}}, // {FieldID: 1, Name: "f1", IsPrimaryKey: false, DataType: schemapb.DataType_VECTOR_FLOAT, TypeParams: []*commonpb.KeyValuePair{{Key: "dim", Value: "128"}}},
}, // },
} // }
schemaBytes, err := proto.Marshal(schema) // schemaBytes, err := proto.Marshal(schema)
assert.Nil(t, err) // assert.Nil(t, err)
_, err = cli.CreateCollection(ctx, &milvuspb.CreateCollectionRequest{ // _, err = cli.CreateCollection(ctx, &milvuspb.CreateCollectionRequest{
Base: &commonpb.MsgBase{ // Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_kCreateCollection, // MsgType: commonpb.MsgType_kCreateCollection,
MsgID: 1, // MsgID: 1,
Timestamp: Timestamp(time.Now().Unix()), // Timestamp: Timestamp(time.Now().Unix()),
SourceID: 1, // SourceID: 1,
}, // },
Schema: schemaBytes, // Schema: schemaBytes,
}) // })
assert.Nil(t, err) // assert.Nil(t, err)
_, err = cli.CreatePartition(ctx, &milvuspb.CreatePartitionRequest{ // _, err = cli.CreatePartition(ctx, &milvuspb.CreatePartitionRequest{
Base: &commonpb.MsgBase{ // Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_kCreatePartition, // MsgType: commonpb.MsgType_kCreatePartition,
MsgID: 2, // MsgID: 2,
Timestamp: Timestamp(time.Now().Unix()), // Timestamp: Timestamp(time.Now().Unix()),
SourceID: 1, // SourceID: 1,
}, // },
CollectionName: collName, // CollectionName: collName,
PartitionName: partitionName, // PartitionName: partitionName,
}) // })
assert.Nil(t, err) // assert.Nil(t, err)
//
resp, err := cli.AssignSegmentID(ctx, &datapb.AssignSegIDRequest{ // resp, err := cli.AssignSegmentID(ctx, &datapb.AssignSegIDRequest{
NodeID: 1, // NodeID: 1,
PeerRole: typeutil.ProxyNodeRole, // PeerRole: typeutil.ProxyNodeRole,
SegIDRequests: []*datapb.SegIDRequest{ // SegIDRequests: []*datapb.SegIDRequest{
{Count: 10000, ChannelName: "0", CollName: collName, PartitionName: partitionName}, // {Count: 10000, ChannelName: "0", CollName: collName, PartitionName: partitionName},
}, // },
}) // })
assert.Nil(t, err) // assert.Nil(t, err)
assignments := resp.GetSegIDAssignments() // assignments := resp.GetSegIDAssignments()
assert.EqualValues(t, 1, len(assignments)) // assert.EqualValues(t, 1, len(assignments))
assert.EqualValues(t, commonpb.ErrorCode_SUCCESS, assignments[0].Status.ErrorCode) // assert.EqualValues(t, commonpb.ErrorCode_SUCCESS, assignments[0].Status.ErrorCode)
assert.EqualValues(t, collName, assignments[0].CollName) // assert.EqualValues(t, collName, assignments[0].CollName)
assert.EqualValues(t, partitionName, assignments[0].PartitionName) // assert.EqualValues(t, partitionName, assignments[0].PartitionName)
assert.EqualValues(t, "0", assignments[0].ChannelName) // assert.EqualValues(t, "0", assignments[0].ChannelName)
assert.EqualValues(t, uint32(10000), assignments[0].Count) // assert.EqualValues(t, uint32(10000), assignments[0].Count)
//
// test stats // // test stats
segID := assignments[0].SegID // segID := assignments[0].SegID
pulsarAddress := Params.PulsarAddress // pulsarAddress := Params.PulsarAddress
msgStream := pulsarms.NewPulsarMsgStream(ctx, 1024) // msgStream := pulsarms.NewPulsarMsgStream(ctx, 1024)
msgStream.SetPulsarClient(pulsarAddress) // msgStream.SetPulsarClient(pulsarAddress)
msgStream.CreatePulsarProducers([]string{Params.QueryNodeStatsChannelName}) // msgStream.CreatePulsarProducers([]string{Params.QueryNodeStatsChannelName})
msgStream.Start() // msgStream.Start()
defer msgStream.Close() // defer msgStream.Close()
//
err = msgStream.Produce(&ms.MsgPack{ // err = msgStream.Produce(&ms.MsgPack{
BeginTs: 102, // BeginTs: 102,
EndTs: 104, // EndTs: 104,
Msgs: []ms.TsMsg{ // Msgs: []ms.TsMsg{
&ms.QueryNodeStatsMsg{ // &ms.QueryNodeStatsMsg{
QueryNodeStats: internalpb2.QueryNodeStats{ // QueryNodeStats: internalpb2.QueryNodeStats{
Base: &commonpb.MsgBase{ // Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_kQueryNodeStats, // MsgType: commonpb.MsgType_kQueryNodeStats,
SourceID: 1, // SourceID: 1,
}, // },
SegStats: []*internalpb2.SegmentStats{ // SegStats: []*internalpb2.SegmentStats{
{SegmentID: segID, MemorySize: 600000000, NumRows: 1000000, RecentlyModified: true}, // {SegmentID: segID, MemorySize: 600000000, NumRows: 1000000, RecentlyModified: true},
}, // },
}, // },
BaseMsg: ms.BaseMsg{ // BaseMsg: ms.BaseMsg{
HashValues: []uint32{0}, // HashValues: []uint32{0},
}, // },
}, // },
}, // },
}) // })
assert.Nil(t, err) // assert.Nil(t, err)
//
time.Sleep(500 * time.Millisecond) // time.Sleep(500 * time.Millisecond)
segMeta, err := svr.metaTable.GetSegmentByID(segID) // segMeta, err := svr.metaTable.GetSegmentByID(segID)
assert.Nil(t, err) // assert.Nil(t, err)
assert.EqualValues(t, 1000000, segMeta.GetNumRows()) // assert.EqualValues(t, 1000000, segMeta.GetNumRows())
assert.EqualValues(t, int64(600000000), segMeta.GetMemSize()) // assert.EqualValues(t, int64(600000000), segMeta.GetMemSize())
//
reqDrop := milvuspb.DropCollectionRequest{ // reqDrop := milvuspb.DropCollectionRequest{
Base: &commonpb.MsgBase{ // Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_kDropCollection, // MsgType: commonpb.MsgType_kDropCollection,
MsgID: 1, // MsgID: 1,
Timestamp: Timestamp(time.Now().Unix()), // Timestamp: Timestamp(time.Now().Unix()),
SourceID: 1, // SourceID: 1,
}, // },
CollectionName: collName, // CollectionName: collName,
} // }
//
// DropCollection // // DropCollection
st, err := cli.DropCollection(ctx, &reqDrop) // st, err := cli.DropCollection(ctx, &reqDrop)
assert.Nil(t, err) // assert.Nil(t, err)
assert.Equal(t, st.ErrorCode, commonpb.ErrorCode_SUCCESS) // assert.Equal(t, st.ErrorCode, commonpb.ErrorCode_SUCCESS)
}) //})
cancel() cancel()
conn.Close() conn.Close()

View File

@ -6,7 +6,6 @@ option go_package="github.com/zilliztech/milvus-distributed/internal/proto/maste
import "common.proto"; import "common.proto";
import "milvus.proto"; import "milvus.proto";
import "internal.proto"; import "internal.proto";
import "data_service.proto";
message IDRequest { message IDRequest {
common.MsgBase base = 1; common.MsgBase base = 1;
@ -109,14 +108,10 @@ service MasterService {
rpc CreateIndex(milvus.CreateIndexRequest) returns (common.Status) {} rpc CreateIndex(milvus.CreateIndexRequest) returns (common.Status) {}
rpc DescribeIndex(milvus.DescribeIndexRequest) returns (milvus.DescribeIndexResponse) {} rpc DescribeIndex(milvus.DescribeIndexRequest) returns (milvus.DescribeIndexResponse) {}
rpc GetIndexState(milvus.IndexStateRequest) returns (milvus.IndexStateResponse) {}
rpc AllocTimestamp(TsoRequest) returns (TsoResponse) {} rpc AllocTimestamp(TsoRequest) returns (TsoResponse) {}
rpc AllocID(IDRequest) returns (IDResponse) {} rpc AllocID(IDRequest) returns (IDResponse) {}
// todo move to data service
rpc AssignSegmentID(data.AssignSegIDRequest) returns (data.AssignSegIDResponse) {}
rpc GetComponentStatesRPC(common.Empty) returns (internal.ComponentStates) {} rpc GetComponentStatesRPC(common.Empty) returns (internal.ComponentStates) {}
rpc GetTimeTickChannelRPC(common.Empty) returns (milvus.StringResponse) {} rpc GetTimeTickChannelRPC(common.Empty) returns (milvus.StringResponse) {}

View File

@ -8,7 +8,6 @@ import (
fmt "fmt" fmt "fmt"
proto "github.com/golang/protobuf/proto" proto "github.com/golang/protobuf/proto"
commonpb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb" commonpb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
datapb "github.com/zilliztech/milvus-distributed/internal/proto/datapb"
internalpb2 "github.com/zilliztech/milvus-distributed/internal/proto/internalpb2" internalpb2 "github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
milvuspb "github.com/zilliztech/milvus-distributed/internal/proto/milvuspb" milvuspb "github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
grpc "google.golang.org/grpc" grpc "google.golang.org/grpc"
@ -242,52 +241,49 @@ func init() {
func init() { proto.RegisterFile("master.proto", fileDescriptor_f9c348dec43a6705) } func init() { proto.RegisterFile("master.proto", fileDescriptor_f9c348dec43a6705) }
var fileDescriptor_f9c348dec43a6705 = []byte{ var fileDescriptor_f9c348dec43a6705 = []byte{
// 715 bytes of a gzipped FileDescriptorProto // 662 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x96, 0xeb, 0x6e, 0xd3, 0x30, 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x95, 0xe9, 0x6e, 0xd3, 0x4c,
0x18, 0x86, 0x77, 0x62, 0x68, 0xdf, 0x7a, 0x98, 0x0c, 0x88, 0xa9, 0x4c, 0x6c, 0x14, 0xb1, 0x75, 0x14, 0x86, 0xbb, 0x7d, 0xfd, 0xd4, 0xd3, 0x2c, 0x68, 0x00, 0x51, 0x99, 0x8a, 0x96, 0x20, 0x41,
0x07, 0x52, 0xb4, 0xdd, 0x00, 0x6b, 0x83, 0xba, 0x4a, 0x4c, 0x9a, 0xd2, 0xf2, 0x83, 0xc1, 0x34, 0xba, 0x90, 0xa0, 0xf6, 0x06, 0x20, 0x31, 0x6a, 0x23, 0x51, 0xa9, 0x72, 0xc2, 0x0f, 0x0a, 0x55,
0x25, 0xa9, 0xd5, 0x5a, 0x24, 0x76, 0x88, 0xdd, 0x0d, 0x76, 0x0b, 0xdc, 0x34, 0x8a, 0x93, 0x38, 0xe5, 0x38, 0x47, 0xc9, 0x08, 0x7b, 0xc6, 0x78, 0xc6, 0x2d, 0xf4, 0xf6, 0xb8, 0x31, 0xe4, 0xf1,
0x49, 0x97, 0xac, 0x41, 0xec, 0xa7, 0xed, 0xd7, 0xcf, 0xfb, 0x9d, 0x14, 0x07, 0x2a, 0xae, 0xc9, 0x9e, 0xda, 0xad, 0x11, 0xf9, 0x39, 0x3e, 0xef, 0x3c, 0xef, 0xd9, 0x92, 0x81, 0x9a, 0x63, 0x0a,
0x05, 0xf6, 0x35, 0xcf, 0x67, 0x82, 0xa1, 0x67, 0x2e, 0x71, 0x6e, 0xa6, 0x3c, 0x5c, 0x69, 0xe1, 0x89, 0x5e, 0xc7, 0xf5, 0xb8, 0xe4, 0xe4, 0xb1, 0x43, 0xed, 0x6b, 0x5f, 0x84, 0xa7, 0x4e, 0x18,
0x51, 0xa3, 0x62, 0x33, 0xd7, 0x65, 0x34, 0xdc, 0x6c, 0x54, 0xd2, 0x92, 0x46, 0x8d, 0x50, 0x81, 0xd2, 0x6a, 0x16, 0x77, 0x1c, 0xce, 0xc2, 0x8f, 0x5a, 0x2d, 0x2b, 0xd1, 0x1a, 0x94, 0x49, 0xf4,
0x7d, 0x6a, 0x3a, 0xd1, 0x1a, 0x8d, 0x4c, 0x61, 0x5e, 0x73, 0xec, 0xdf, 0x10, 0x1b, 0x87, 0x7b, 0x98, 0x69, 0x87, 0xe7, 0xd6, 0x10, 0x36, 0x06, 0xba, 0x81, 0x3f, 0x7c, 0x14, 0x92, 0xbc, 0x83,
0xcd, 0x01, 0xac, 0xf5, 0x75, 0x03, 0xff, 0x9c, 0x62, 0x2e, 0xd0, 0x07, 0x58, 0xb1, 0x4c, 0x8e, 0xb5, 0xb1, 0x29, 0x70, 0x6b, 0x79, 0x77, 0xb9, 0xbd, 0x79, 0xb4, 0xdd, 0xc9, 0xc1, 0x23, 0xe8,
0x37, 0x17, 0x77, 0x16, 0x5b, 0xeb, 0xc7, 0x5b, 0x5a, 0xc6, 0x30, 0x32, 0x3a, 0xe7, 0xe3, 0x8e, 0x99, 0x98, 0xf6, 0x4c, 0x81, 0x86, 0x52, 0x92, 0x27, 0xf0, 0x9f, 0xc5, 0x7d, 0x26, 0xb7, 0x56,
0xc9, 0xb1, 0x21, 0x95, 0xe8, 0x39, 0x3c, 0xb1, 0xd9, 0x94, 0x8a, 0xcd, 0xa5, 0x9d, 0xc5, 0x56, 0x76, 0x97, 0xdb, 0x75, 0x23, 0x3c, 0xb4, 0xa6, 0x00, 0x01, 0x54, 0xb8, 0x9c, 0x09, 0x24, 0xc7,
0xd5, 0x08, 0x17, 0xcd, 0x31, 0x40, 0x00, 0xe5, 0x1e, 0xa3, 0x1c, 0xa3, 0x13, 0x58, 0xe5, 0xc2, 0xb0, 0x2e, 0xa4, 0x29, 0x7d, 0x11, 0x71, 0x9f, 0x17, 0x72, 0x87, 0x4a, 0x62, 0x44, 0x52, 0xd2,
0x14, 0x53, 0x1e, 0x71, 0x5f, 0xe5, 0x72, 0x07, 0x52, 0x62, 0x44, 0x52, 0x54, 0x83, 0xa5, 0xbe, 0x80, 0x95, 0x81, 0xae, 0xa8, 0xab, 0xc6, 0xca, 0x40, 0x4f, 0x8d, 0x56, 0xb3, 0x46, 0x23, 0x80,
0x2e, 0xa9, 0xcb, 0xc6, 0x52, 0x5f, 0x4f, 0x8c, 0x96, 0xd3, 0x46, 0x43, 0x80, 0x21, 0x67, 0x8f, 0x91, 0xe0, 0x0b, 0x48, 0x3f, 0x47, 0xbd, 0x86, 0x4d, 0x45, 0xfd, 0x97, 0xfc, 0xb7, 0x61, 0x43,
0x10, 0x7e, 0x86, 0x7a, 0x03, 0xeb, 0x92, 0xfa, 0x3f, 0xf1, 0x6f, 0xc1, 0x9a, 0x20, 0x2e, 0xe6, 0x52, 0x07, 0x85, 0x34, 0x1d, 0x57, 0x95, 0xb1, 0x66, 0xa4, 0x1f, 0x8a, 0x7d, 0x8f, 0x7e, 0x37,
0xc2, 0x74, 0x3d, 0x99, 0xc6, 0x8a, 0x91, 0x6c, 0xe4, 0xfb, 0x1e, 0xff, 0xd9, 0x80, 0xea, 0xb9, 0xa0, 0x7e, 0xa6, 0x46, 0x38, 0x44, 0xef, 0x9a, 0x5a, 0x48, 0xae, 0xe0, 0x51, 0xdf, 0x43, 0x53,
0x6c, 0xeb, 0x20, 0xec, 0x11, 0xba, 0x86, 0x8d, 0xae, 0x8f, 0x4d, 0x81, 0xbb, 0xcc, 0x71, 0xb0, 0x62, 0x9f, 0xdb, 0x36, 0x5a, 0x92, 0x72, 0x46, 0x0e, 0xf3, 0xf6, 0xd1, 0x61, 0x5e, 0x16, 0xf5,
0x2d, 0x08, 0xa3, 0xe8, 0x28, 0x6b, 0x1f, 0x2d, 0x66, 0x65, 0x51, 0x4d, 0x1a, 0x0f, 0x05, 0xdb, 0x44, 0xbb, 0x2f, 0xd9, 0xd6, 0x12, 0xf9, 0x0a, 0x0d, 0xdd, 0xe3, 0x6e, 0x06, 0xbf, 0x5f, 0x88,
0x5c, 0x40, 0xdf, 0xa0, 0xa6, 0xfb, 0xcc, 0x4b, 0xe1, 0x0f, 0x72, 0xf1, 0x59, 0x51, 0x49, 0xf8, 0xcf, 0x8b, 0x2a, 0xc2, 0xaf, 0xa0, 0x7e, 0x6a, 0x8a, 0x0c, 0x7b, 0xaf, 0x90, 0x9d, 0xd3, 0xc4,
0x35, 0x54, 0xcf, 0x4c, 0x9e, 0x62, 0xef, 0xe7, 0xb2, 0x33, 0x9a, 0x18, 0xfd, 0x26, 0x57, 0xda, 0xe8, 0x97, 0x85, 0xd2, 0x1e, 0xe7, 0x76, 0x3c, 0x97, 0xd6, 0x12, 0xb9, 0x01, 0xa2, 0xa3, 0xb0,
0x61, 0xcc, 0x89, 0xfb, 0xd2, 0x5c, 0x40, 0xb7, 0x80, 0x74, 0xcc, 0x6d, 0x9f, 0x58, 0xe9, 0x02, 0x3c, 0x3a, 0xce, 0x36, 0xa8, 0x53, 0x5c, 0xc1, 0x1d, 0x61, 0x6c, 0xd5, 0xad, 0xac, 0x4f, 0x8c,
0x69, 0xf9, 0x19, 0xdc, 0x13, 0xc6, 0x56, 0xed, 0xd2, 0x7a, 0x65, 0xec, 0x40, 0x7d, 0x30, 0x61, 0x6d, 0x68, 0x0e, 0x67, 0xfc, 0x26, 0x8d, 0x89, 0x92, 0xbe, 0xe5, 0x55, 0xb1, 0xe3, 0x41, 0x25,
0xb7, 0xc9, 0x19, 0x2f, 0xa8, 0x5b, 0x56, 0x15, 0x3b, 0x1e, 0x96, 0xd2, 0x2a, 0xb7, 0x2b, 0xa8, 0x6d, 0xe2, 0x76, 0x09, 0xcd, 0x70, 0xbc, 0xe7, 0xa6, 0x27, 0xa9, 0xaa, 0xf1, 0xe0, 0x9e, 0x25,
0x87, 0xed, 0xbd, 0x30, 0x7d, 0x41, 0x64, 0x8e, 0x87, 0x0f, 0x0c, 0x81, 0x52, 0x95, 0x6c, 0xd3, 0x48, 0x54, 0x15, 0xc7, 0xf4, 0x05, 0xea, 0xc1, 0x78, 0x53, 0xf8, 0x5e, 0xe9, 0x0a, 0xfc, 0x2d,
0x57, 0xa8, 0x06, 0xed, 0x4d, 0xe0, 0xfb, 0x85, 0x23, 0xf0, 0xaf, 0xe8, 0x2b, 0xa8, 0x9c, 0x99, 0xfa, 0x12, 0x6a, 0xa7, 0xa6, 0x48, 0xc9, 0xed, 0xb2, 0x05, 0xb8, 0x03, 0xae, 0x34, 0x7f, 0x0a,
0x3c, 0x21, 0xb7, 0x8a, 0x06, 0xe0, 0x1e, 0xb8, 0x54, 0xff, 0x09, 0xd4, 0x82, 0xa2, 0xa9, 0xcb, 0x8d, 0xa0, 0x69, 0xc9, 0x65, 0x51, 0x92, 0x7a, 0x4e, 0x14, 0x3b, 0xec, 0x57, 0x91, 0x26, 0x56,
0xbc, 0x20, 0xf4, 0x8c, 0x28, 0x76, 0x38, 0x28, 0x23, 0x55, 0x56, 0x14, 0xea, 0xf1, 0x44, 0x0c, 0x0c, 0x9a, 0xf1, 0x46, 0x0c, 0x71, 0xea, 0x20, 0x93, 0x25, 0x33, 0x98, 0x53, 0xc5, 0x6e, 0x87,
0xf0, 0xd8, 0xc5, 0x54, 0x14, 0xf4, 0x60, 0x46, 0x15, 0xbb, 0x1d, 0x95, 0x13, 0x2b, 0x3f, 0x1b, 0xd5, 0xc4, 0x89, 0x9f, 0x05, 0xb5, 0x20, 0x95, 0x28, 0x20, 0xc8, 0x9b, 0xd2, 0x6c, 0xe7, 0x8c,
0x2a, 0x41, 0x28, 0xd1, 0x01, 0x47, 0x7b, 0x85, 0xd1, 0xce, 0x18, 0xb5, 0xe6, 0x0b, 0x95, 0xc9, 0xda, 0x0f, 0x0b, 0x13, 0x93, 0xcf, 0xb0, 0x19, 0xae, 0xcc, 0x80, 0x4d, 0xf0, 0x67, 0x89, 0x47,
0x17, 0x58, 0x0f, 0x47, 0xa6, 0x4f, 0x47, 0xf8, 0x57, 0x81, 0x47, 0x4a, 0x51, 0xb2, 0xeb, 0x13, 0x46, 0x51, 0x71, 0xea, 0x33, 0xa8, 0xc7, 0x85, 0x85, 0xe0, 0xbd, 0x7b, 0x8b, 0xcf, 0xa1, 0xf7,
0xa8, 0xc6, 0x89, 0x85, 0xe0, 0xfd, 0x07, 0x93, 0xcf, 0xa0, 0x0f, 0xca, 0x48, 0x55, 0x02, 0x16, 0xab, 0x48, 0x33, 0x05, 0x34, 0x3e, 0xd8, 0x36, 0xb7, 0x46, 0xc9, 0x3f, 0xeb, 0x4e, 0xa7, 0xe0,
0x54, 0x7b, 0x58, 0xc8, 0xdd, 0xc0, 0x1d, 0xa3, 0xdd, 0xdc, 0xeb, 0x89, 0x20, 0xb6, 0xd9, 0x9b, 0x45, 0xec, 0xa4, 0x8f, 0x84, 0xb6, 0x5b, 0x2e, 0x48, 0xb0, 0x9f, 0xe0, 0x7f, 0x85, 0x1d, 0xe8,
0xab, 0x4b, 0x15, 0xa9, 0x76, 0xea, 0x38, 0xcc, 0x1e, 0xaa, 0xaf, 0xf7, 0xb6, 0x96, 0xf3, 0x12, 0xe4, 0x45, 0xa1, 0x3c, 0x79, 0x32, 0xb5, 0x9d, 0xd2, 0x78, 0xe6, 0xe7, 0xfb, 0xf4, 0x04, 0x65,
0x6b, 0xc9, 0x43, 0xd4, 0xd8, 0x29, 0x16, 0x28, 0xec, 0x67, 0x78, 0x2a, 0xb1, 0x7d, 0x1d, 0xbd, 0x9f, 0x3b, 0x2e, 0x67, 0xc8, 0x64, 0xd0, 0x26, 0x14, 0xc6, 0x79, 0x9f, 0x68, 0x85, 0x6d, 0xfc,
0xce, 0x95, 0xab, 0x67, 0xb9, 0xb1, 0x5d, 0x78, 0x9e, 0x2a, 0x44, 0xfd, 0x94, 0x73, 0x32, 0xa6, 0xe8, 0xb8, 0xf2, 0x97, 0xf6, 0x3a, 0x1f, 0x4b, 0x5e, 0xed, 0x39, 0x4c, 0x6b, 0x89, 0x5c, 0x28,
0x51, 0x93, 0xfb, 0x3a, 0x7a, 0x97, 0xbd, 0x15, 0xbc, 0xfd, 0x9a, 0xd2, 0x24, 0xf0, 0xdd, 0x79, 0x7c, 0xd0, 0x81, 0x11, 0xb5, 0xbe, 0xf7, 0x67, 0x26, 0x63, 0x68, 0x3f, 0x84, 0x7f, 0x55, 0xbc,
0xb2, 0xd4, 0x67, 0xe8, 0x45, 0x0f, 0x8b, 0x2e, 0x73, 0x3d, 0x46, 0x31, 0x15, 0xb2, 0x4e, 0xdc, 0x25, 0xd2, 0xa3, 0x6c, 0x9a, 0x49, 0x7d, 0x04, 0xcd, 0x13, 0x94, 0xfa, 0x64, 0xb1, 0xd4, 0x6f,
0xb8, 0xe8, 0xa2, 0x46, 0xee, 0x38, 0x7c, 0x72, 0x3d, 0xf1, 0x7b, 0x16, 0xaf, 0xfe, 0x48, 0x66, 0xf0, 0xec, 0x04, 0x55, 0x01, 0x54, 0x48, 0x6a, 0x89, 0x85, 0xd2, 0x7b, 0xbd, 0x8b, 0xf7, 0x53,
0x30, 0xcd, 0x05, 0x74, 0x29, 0xf1, 0x41, 0x95, 0x87, 0xc4, 0xfe, 0xd1, 0x9d, 0x98, 0x94, 0x62, 0x2a, 0x67, 0xfe, 0x38, 0xb8, 0xdd, 0xbd, 0xa5, 0xb6, 0x4d, 0x6f, 0x25, 0x5a, 0xb3, 0x6e, 0x78,
0x67, 0x1e, 0xfe, 0x6d, 0xfe, 0xb4, 0x0b, 0x9f, 0xd0, 0x71, 0x2a, 0xf4, 0x21, 0xd4, 0x7b, 0x58, 0xe1, 0xed, 0x84, 0x0a, 0xe9, 0xd1, 0xb1, 0x2f, 0x71, 0xd2, 0x8d, 0xdb, 0xda, 0x55, 0xc8, 0x6e,
0xe8, 0xa3, 0xc7, 0xa5, 0x7e, 0x87, 0x97, 0x3d, 0x2c, 0x13, 0x20, 0x5c, 0x10, 0x9b, 0x3f, 0x2a, 0x38, 0x3d, 0x77, 0x3c, 0x5e, 0x57, 0xe7, 0xe3, 0x3f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x41, 0x9e,
0xbd, 0xd3, 0xb9, 0xfc, 0x38, 0x26, 0x62, 0x32, 0xb5, 0x82, 0xdb, 0xed, 0x3b, 0xe2, 0x38, 0xe4, 0x3e, 0x16, 0x6d, 0x09, 0x00, 0x00,
0x4e, 0x60, 0x7b, 0xd2, 0x0e, 0x2f, 0xbc, 0x1f, 0x11, 0x2e, 0x7c, 0x62, 0x4d, 0x05, 0x1e, 0xb5,
0xe3, 0xb2, 0xb6, 0x25, 0xb2, 0x1d, 0x4e, 0x88, 0x67, 0x59, 0xab, 0x72, 0x7d, 0xf2, 0x37, 0x00,
0x00, 0xff, 0xff, 0x0a, 0x65, 0x49, 0xe7, 0x49, 0x0a, 0x00, 0x00,
} }
// Reference imports to suppress errors if they are not otherwise used. // Reference imports to suppress errors if they are not otherwise used.
@ -361,11 +357,8 @@ type MasterServiceClient interface {
ShowSegments(ctx context.Context, in *milvuspb.ShowSegmentRequest, opts ...grpc.CallOption) (*milvuspb.ShowSegmentResponse, error) ShowSegments(ctx context.Context, in *milvuspb.ShowSegmentRequest, opts ...grpc.CallOption) (*milvuspb.ShowSegmentResponse, error)
CreateIndex(ctx context.Context, in *milvuspb.CreateIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) CreateIndex(ctx context.Context, in *milvuspb.CreateIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
DescribeIndex(ctx context.Context, in *milvuspb.DescribeIndexRequest, opts ...grpc.CallOption) (*milvuspb.DescribeIndexResponse, error) DescribeIndex(ctx context.Context, in *milvuspb.DescribeIndexRequest, opts ...grpc.CallOption) (*milvuspb.DescribeIndexResponse, error)
GetIndexState(ctx context.Context, in *milvuspb.IndexStateRequest, opts ...grpc.CallOption) (*milvuspb.IndexStateResponse, error)
AllocTimestamp(ctx context.Context, in *TsoRequest, opts ...grpc.CallOption) (*TsoResponse, error) AllocTimestamp(ctx context.Context, in *TsoRequest, opts ...grpc.CallOption) (*TsoResponse, error)
AllocID(ctx context.Context, in *IDRequest, opts ...grpc.CallOption) (*IDResponse, error) AllocID(ctx context.Context, in *IDRequest, opts ...grpc.CallOption) (*IDResponse, error)
// todo move to data service
AssignSegmentID(ctx context.Context, in *datapb.AssignSegIDRequest, opts ...grpc.CallOption) (*datapb.AssignSegIDResponse, error)
GetComponentStatesRPC(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*internalpb2.ComponentStates, error) GetComponentStatesRPC(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*internalpb2.ComponentStates, error)
GetTimeTickChannelRPC(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) GetTimeTickChannelRPC(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*milvuspb.StringResponse, error)
GetDdChannelRPC(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) GetDdChannelRPC(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*milvuspb.StringResponse, error)
@ -497,15 +490,6 @@ func (c *masterServiceClient) DescribeIndex(ctx context.Context, in *milvuspb.De
return out, nil return out, nil
} }
func (c *masterServiceClient) GetIndexState(ctx context.Context, in *milvuspb.IndexStateRequest, opts ...grpc.CallOption) (*milvuspb.IndexStateResponse, error) {
out := new(milvuspb.IndexStateResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.master.MasterService/GetIndexState", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *masterServiceClient) AllocTimestamp(ctx context.Context, in *TsoRequest, opts ...grpc.CallOption) (*TsoResponse, error) { func (c *masterServiceClient) AllocTimestamp(ctx context.Context, in *TsoRequest, opts ...grpc.CallOption) (*TsoResponse, error) {
out := new(TsoResponse) out := new(TsoResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.master.MasterService/AllocTimestamp", in, out, opts...) err := c.cc.Invoke(ctx, "/milvus.proto.master.MasterService/AllocTimestamp", in, out, opts...)
@ -524,15 +508,6 @@ func (c *masterServiceClient) AllocID(ctx context.Context, in *IDRequest, opts .
return out, nil return out, nil
} }
func (c *masterServiceClient) AssignSegmentID(ctx context.Context, in *datapb.AssignSegIDRequest, opts ...grpc.CallOption) (*datapb.AssignSegIDResponse, error) {
out := new(datapb.AssignSegIDResponse)
err := c.cc.Invoke(ctx, "/milvus.proto.master.MasterService/AssignSegmentID", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *masterServiceClient) GetComponentStatesRPC(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*internalpb2.ComponentStates, error) { func (c *masterServiceClient) GetComponentStatesRPC(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*internalpb2.ComponentStates, error) {
out := new(internalpb2.ComponentStates) out := new(internalpb2.ComponentStates)
err := c.cc.Invoke(ctx, "/milvus.proto.master.MasterService/GetComponentStatesRPC", in, out, opts...) err := c.cc.Invoke(ctx, "/milvus.proto.master.MasterService/GetComponentStatesRPC", in, out, opts...)
@ -630,11 +605,8 @@ type MasterServiceServer interface {
ShowSegments(context.Context, *milvuspb.ShowSegmentRequest) (*milvuspb.ShowSegmentResponse, error) ShowSegments(context.Context, *milvuspb.ShowSegmentRequest) (*milvuspb.ShowSegmentResponse, error)
CreateIndex(context.Context, *milvuspb.CreateIndexRequest) (*commonpb.Status, error) CreateIndex(context.Context, *milvuspb.CreateIndexRequest) (*commonpb.Status, error)
DescribeIndex(context.Context, *milvuspb.DescribeIndexRequest) (*milvuspb.DescribeIndexResponse, error) DescribeIndex(context.Context, *milvuspb.DescribeIndexRequest) (*milvuspb.DescribeIndexResponse, error)
GetIndexState(context.Context, *milvuspb.IndexStateRequest) (*milvuspb.IndexStateResponse, error)
AllocTimestamp(context.Context, *TsoRequest) (*TsoResponse, error) AllocTimestamp(context.Context, *TsoRequest) (*TsoResponse, error)
AllocID(context.Context, *IDRequest) (*IDResponse, error) AllocID(context.Context, *IDRequest) (*IDResponse, error)
// todo move to data service
AssignSegmentID(context.Context, *datapb.AssignSegIDRequest) (*datapb.AssignSegIDResponse, error)
GetComponentStatesRPC(context.Context, *commonpb.Empty) (*internalpb2.ComponentStates, error) GetComponentStatesRPC(context.Context, *commonpb.Empty) (*internalpb2.ComponentStates, error)
GetTimeTickChannelRPC(context.Context, *commonpb.Empty) (*milvuspb.StringResponse, error) GetTimeTickChannelRPC(context.Context, *commonpb.Empty) (*milvuspb.StringResponse, error)
GetDdChannelRPC(context.Context, *commonpb.Empty) (*milvuspb.StringResponse, error) GetDdChannelRPC(context.Context, *commonpb.Empty) (*milvuspb.StringResponse, error)
@ -684,18 +656,12 @@ func (*UnimplementedMasterServiceServer) CreateIndex(ctx context.Context, req *m
func (*UnimplementedMasterServiceServer) DescribeIndex(ctx context.Context, req *milvuspb.DescribeIndexRequest) (*milvuspb.DescribeIndexResponse, error) { func (*UnimplementedMasterServiceServer) DescribeIndex(ctx context.Context, req *milvuspb.DescribeIndexRequest) (*milvuspb.DescribeIndexResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method DescribeIndex not implemented") return nil, status.Errorf(codes.Unimplemented, "method DescribeIndex not implemented")
} }
func (*UnimplementedMasterServiceServer) GetIndexState(ctx context.Context, req *milvuspb.IndexStateRequest) (*milvuspb.IndexStateResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetIndexState not implemented")
}
func (*UnimplementedMasterServiceServer) AllocTimestamp(ctx context.Context, req *TsoRequest) (*TsoResponse, error) { func (*UnimplementedMasterServiceServer) AllocTimestamp(ctx context.Context, req *TsoRequest) (*TsoResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method AllocTimestamp not implemented") return nil, status.Errorf(codes.Unimplemented, "method AllocTimestamp not implemented")
} }
func (*UnimplementedMasterServiceServer) AllocID(ctx context.Context, req *IDRequest) (*IDResponse, error) { func (*UnimplementedMasterServiceServer) AllocID(ctx context.Context, req *IDRequest) (*IDResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method AllocID not implemented") return nil, status.Errorf(codes.Unimplemented, "method AllocID not implemented")
} }
func (*UnimplementedMasterServiceServer) AssignSegmentID(ctx context.Context, req *datapb.AssignSegIDRequest) (*datapb.AssignSegIDResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method AssignSegmentID not implemented")
}
func (*UnimplementedMasterServiceServer) GetComponentStatesRPC(ctx context.Context, req *commonpb.Empty) (*internalpb2.ComponentStates, error) { func (*UnimplementedMasterServiceServer) GetComponentStatesRPC(ctx context.Context, req *commonpb.Empty) (*internalpb2.ComponentStates, error) {
return nil, status.Errorf(codes.Unimplemented, "method GetComponentStatesRPC not implemented") return nil, status.Errorf(codes.Unimplemented, "method GetComponentStatesRPC not implemented")
} }
@ -947,24 +913,6 @@ func _MasterService_DescribeIndex_Handler(srv interface{}, ctx context.Context,
return interceptor(ctx, in, info, handler) return interceptor(ctx, in, info, handler)
} }
func _MasterService_GetIndexState_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(milvuspb.IndexStateRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(MasterServiceServer).GetIndexState(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/milvus.proto.master.MasterService/GetIndexState",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(MasterServiceServer).GetIndexState(ctx, req.(*milvuspb.IndexStateRequest))
}
return interceptor(ctx, in, info, handler)
}
func _MasterService_AllocTimestamp_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { func _MasterService_AllocTimestamp_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(TsoRequest) in := new(TsoRequest)
if err := dec(in); err != nil { if err := dec(in); err != nil {
@ -1001,24 +949,6 @@ func _MasterService_AllocID_Handler(srv interface{}, ctx context.Context, dec fu
return interceptor(ctx, in, info, handler) return interceptor(ctx, in, info, handler)
} }
func _MasterService_AssignSegmentID_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(datapb.AssignSegIDRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(MasterServiceServer).AssignSegmentID(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/milvus.proto.master.MasterService/AssignSegmentID",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(MasterServiceServer).AssignSegmentID(ctx, req.(*datapb.AssignSegIDRequest))
}
return interceptor(ctx, in, info, handler)
}
func _MasterService_GetComponentStatesRPC_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { func _MasterService_GetComponentStatesRPC_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(commonpb.Empty) in := new(commonpb.Empty)
if err := dec(in); err != nil { if err := dec(in); err != nil {
@ -1147,10 +1077,6 @@ var _MasterService_serviceDesc = grpc.ServiceDesc{
MethodName: "DescribeIndex", MethodName: "DescribeIndex",
Handler: _MasterService_DescribeIndex_Handler, Handler: _MasterService_DescribeIndex_Handler,
}, },
{
MethodName: "GetIndexState",
Handler: _MasterService_GetIndexState_Handler,
},
{ {
MethodName: "AllocTimestamp", MethodName: "AllocTimestamp",
Handler: _MasterService_AllocTimestamp_Handler, Handler: _MasterService_AllocTimestamp_Handler,
@ -1159,10 +1085,6 @@ var _MasterService_serviceDesc = grpc.ServiceDesc{
MethodName: "AllocID", MethodName: "AllocID",
Handler: _MasterService_AllocID_Handler, Handler: _MasterService_AllocID_Handler,
}, },
{
MethodName: "AssignSegmentID",
Handler: _MasterService_AssignSegmentID_Handler,
},
{ {
MethodName: "GetComponentStatesRPC", MethodName: "GetComponentStatesRPC",
Handler: _MasterService_GetComponentStatesRPC_Handler, Handler: _MasterService_GetComponentStatesRPC_Handler,

View File

@ -1,66 +0,0 @@
package proxynode
import (
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/indexpb"
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
)
type MasterClientInterface interface {
Init() error
Start() error
Stop() error
CreateCollection(in *milvuspb.CreateCollectionRequest) (*commonpb.Status, error)
DropCollection(in *milvuspb.DropCollectionRequest) (*commonpb.Status, error)
HasCollection(in *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error)
DescribeCollection(in *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error)
ShowCollections(in *milvuspb.ShowCollectionRequest) (*milvuspb.ShowCollectionResponse, error)
CreatePartition(in *milvuspb.CreatePartitionRequest) (*commonpb.Status, error)
DropPartition(in *milvuspb.DropPartitionRequest) (*commonpb.Status, error)
HasPartition(in *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error)
ShowPartitions(in *milvuspb.ShowPartitionRequest) (*milvuspb.ShowPartitionResponse, error)
CreateIndex(in *milvuspb.CreateIndexRequest) (*commonpb.Status, error)
DescribeIndex(in *milvuspb.DescribeIndexRequest) (*milvuspb.DescribeIndexResponse, error)
}
type IndexServiceClient interface {
Init() error
Start() error
Stop() error
GetIndexStates(req *indexpb.IndexStatesRequest) (*indexpb.IndexStatesResponse, error)
}
type QueryServiceClient interface {
Init() error
Start() error
Stop() error
GetSearchChannelNames() ([]string, error)
GetSearchResultChannelNames() ([]string, error)
}
type DataServiceClient interface {
Init() error
Start() error
Stop() error
GetInsertChannelNames() ([]string, error)
}
func (node *NodeImpl) SetMasterClient(cli MasterClientInterface) {
node.masterClient = cli
}
func (node *NodeImpl) SetIndexServiceClient(cli IndexServiceClient) {
node.indexServiceClient = cli
}
func (node *NodeImpl) SetQueryServiceClient(cli QueryServiceClient) {
node.queryServiceClient = cli
}
func (node *NodeImpl) SetDataServiceClient(cli DataServiceClient) {
node.dataServiceClient = cli
}

View File

@ -20,12 +20,17 @@ const (
reqTimeoutInterval = time.Second * 10 reqTimeoutInterval = time.Second * 10
) )
func (node *NodeImpl) UpdateStateCode(code internalpb2.StateCode) {
node.stateCode = code
}
func (node *NodeImpl) InvalidateCollectionMetaCache(ctx context.Context, request *proxypb.InvalidateCollMetaCacheRequest) (*commonpb.Status, error) { func (node *NodeImpl) InvalidateCollectionMetaCache(ctx context.Context, request *proxypb.InvalidateCollMetaCacheRequest) (*commonpb.Status, error) {
panic("implement me") panic("implement me")
} }
func (node *NodeImpl) CreateCollection(ctx context.Context, request *milvuspb.CreateCollectionRequest) (*commonpb.Status, error) { func (node *NodeImpl) CreateCollection(request *milvuspb.CreateCollectionRequest) (*commonpb.Status, error) {
log.Println("create collection: ", request) log.Println("create collection: ", request)
ctx := context.Background()
cct := &CreateCollectionTask{ cct := &CreateCollectionTask{
Condition: NewTaskCondition(ctx), Condition: NewTaskCondition(ctx),
CreateCollectionRequest: request, CreateCollectionRequest: request,
@ -62,8 +67,9 @@ func (node *NodeImpl) CreateCollection(ctx context.Context, request *milvuspb.Cr
return cct.result, nil return cct.result, nil
} }
func (node *NodeImpl) DropCollection(ctx context.Context, request *milvuspb.DropCollectionRequest) (*commonpb.Status, error) { func (node *NodeImpl) DropCollection(request *milvuspb.DropCollectionRequest) (*commonpb.Status, error) {
log.Println("drop collection: ", request) log.Println("drop collection: ", request)
ctx := context.Background()
dct := &DropCollectionTask{ dct := &DropCollectionTask{
Condition: NewTaskCondition(ctx), Condition: NewTaskCondition(ctx),
DropCollectionRequest: request, DropCollectionRequest: request,
@ -100,8 +106,9 @@ func (node *NodeImpl) DropCollection(ctx context.Context, request *milvuspb.Drop
return dct.result, nil return dct.result, nil
} }
func (node *NodeImpl) HasCollection(ctx context.Context, request *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error) { func (node *NodeImpl) HasCollection(request *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error) {
log.Println("has collection: ", request) log.Println("has collection: ", request)
ctx := context.Background()
hct := &HasCollectionTask{ hct := &HasCollectionTask{
Condition: NewTaskCondition(ctx), Condition: NewTaskCondition(ctx),
HasCollectionRequest: request, HasCollectionRequest: request,
@ -142,16 +149,17 @@ func (node *NodeImpl) HasCollection(ctx context.Context, request *milvuspb.HasCo
return hct.result, nil return hct.result, nil
} }
func (node *NodeImpl) LoadCollection(ctx context.Context, request *milvuspb.LoadCollectionRequest) (*commonpb.Status, error) { func (node *NodeImpl) LoadCollection(request *milvuspb.LoadCollectionRequest) (*commonpb.Status, error) {
panic("implement me") panic("implement me")
} }
func (node *NodeImpl) ReleaseCollection(ctx context.Context, request *milvuspb.ReleaseCollectionRequest) (*commonpb.Status, error) { func (node *NodeImpl) ReleaseCollection(request *milvuspb.ReleaseCollectionRequest) (*commonpb.Status, error) {
panic("implement me") panic("implement me")
} }
func (node *NodeImpl) DescribeCollection(ctx context.Context, request *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) { func (node *NodeImpl) DescribeCollection(request *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) {
log.Println("describe collection: ", request) log.Println("describe collection: ", request)
ctx := context.Background()
dct := &DescribeCollectionTask{ dct := &DescribeCollectionTask{
Condition: NewTaskCondition(ctx), Condition: NewTaskCondition(ctx),
DescribeCollectionRequest: request, DescribeCollectionRequest: request,
@ -192,12 +200,13 @@ func (node *NodeImpl) DescribeCollection(ctx context.Context, request *milvuspb.
return dct.result, nil return dct.result, nil
} }
func (node *NodeImpl) GetCollectionStatistics(ctx context.Context, request *milvuspb.CollectionStatsRequest) (*milvuspb.CollectionStatsResponse, error) { func (node *NodeImpl) GetCollectionStatistics(request *milvuspb.CollectionStatsRequest) (*milvuspb.CollectionStatsResponse, error) {
panic("implement me") panic("implement me")
} }
func (node *NodeImpl) ShowCollections(ctx context.Context, request *milvuspb.ShowCollectionRequest) (*milvuspb.ShowCollectionResponse, error) { func (node *NodeImpl) ShowCollections(request *milvuspb.ShowCollectionRequest) (*milvuspb.ShowCollectionResponse, error) {
log.Println("show collections") log.Println("show collections")
ctx := context.Background()
sct := &ShowCollectionsTask{ sct := &ShowCollectionsTask{
Condition: NewTaskCondition(ctx), Condition: NewTaskCondition(ctx),
ShowCollectionRequest: request, ShowCollectionRequest: request,
@ -238,8 +247,9 @@ func (node *NodeImpl) ShowCollections(ctx context.Context, request *milvuspb.Sho
return sct.result, nil return sct.result, nil
} }
func (node *NodeImpl) CreatePartition(ctx context.Context, request *milvuspb.CreatePartitionRequest) (*commonpb.Status, error) { func (node *NodeImpl) CreatePartition(request *milvuspb.CreatePartitionRequest) (*commonpb.Status, error) {
log.Println("create partition", request) log.Println("create partition", request)
ctx := context.Background()
cpt := &CreatePartitionTask{ cpt := &CreatePartitionTask{
Condition: NewTaskCondition(ctx), Condition: NewTaskCondition(ctx),
CreatePartitionRequest: request, CreatePartitionRequest: request,
@ -276,8 +286,9 @@ func (node *NodeImpl) CreatePartition(ctx context.Context, request *milvuspb.Cre
return cpt.result, nil return cpt.result, nil
} }
func (node *NodeImpl) DropPartition(ctx context.Context, request *milvuspb.DropPartitionRequest) (*commonpb.Status, error) { func (node *NodeImpl) DropPartition(request *milvuspb.DropPartitionRequest) (*commonpb.Status, error) {
log.Println("drop partition: ", request) log.Println("drop partition: ", request)
ctx := context.Background()
dpt := &DropPartitionTask{ dpt := &DropPartitionTask{
Condition: NewTaskCondition(ctx), Condition: NewTaskCondition(ctx),
DropPartitionRequest: request, DropPartitionRequest: request,
@ -315,8 +326,9 @@ func (node *NodeImpl) DropPartition(ctx context.Context, request *milvuspb.DropP
return dpt.result, nil return dpt.result, nil
} }
func (node *NodeImpl) HasPartition(ctx context.Context, request *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error) { func (node *NodeImpl) HasPartition(request *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error) {
log.Println("has partition: ", request) log.Println("has partition: ", request)
ctx := context.Background()
hpt := &HasPartitionTask{ hpt := &HasPartitionTask{
Condition: NewTaskCondition(ctx), Condition: NewTaskCondition(ctx),
HasPartitionRequest: request, HasPartitionRequest: request,
@ -360,20 +372,21 @@ func (node *NodeImpl) HasPartition(ctx context.Context, request *milvuspb.HasPar
return hpt.result, nil return hpt.result, nil
} }
func (node *NodeImpl) LoadPartitions(ctx context.Context, request *milvuspb.LoadPartitonRequest) (*commonpb.Status, error) { func (node *NodeImpl) LoadPartitions(request *milvuspb.LoadPartitonRequest) (*commonpb.Status, error) {
panic("implement me") panic("implement me")
} }
func (node *NodeImpl) ReleasePartitions(ctx context.Context, request *milvuspb.ReleasePartitionRequest) (*commonpb.Status, error) { func (node *NodeImpl) ReleasePartitions(request *milvuspb.ReleasePartitionRequest) (*commonpb.Status, error) {
panic("implement me") panic("implement me")
} }
func (node *NodeImpl) GetPartitionStatistics(ctx context.Context, request *milvuspb.PartitionStatsRequest) (*milvuspb.PartitionStatsResponse, error) { func (node *NodeImpl) GetPartitionStatistics(request *milvuspb.PartitionStatsRequest) (*milvuspb.PartitionStatsResponse, error) {
panic("implement me") panic("implement me")
} }
func (node *NodeImpl) ShowPartitions(ctx context.Context, request *milvuspb.ShowPartitionRequest) (*milvuspb.ShowPartitionResponse, error) { func (node *NodeImpl) ShowPartitions(request *milvuspb.ShowPartitionRequest) (*milvuspb.ShowPartitionResponse, error) {
log.Println("show partitions: ", request) log.Println("show partitions: ", request)
ctx := context.Background()
spt := &ShowPartitionsTask{ spt := &ShowPartitionsTask{
Condition: NewTaskCondition(ctx), Condition: NewTaskCondition(ctx),
ShowPartitionRequest: request, ShowPartitionRequest: request,
@ -416,8 +429,9 @@ func (node *NodeImpl) ShowPartitions(ctx context.Context, request *milvuspb.Show
return spt.result, nil return spt.result, nil
} }
func (node *NodeImpl) CreateIndex(ctx context.Context, request *milvuspb.CreateIndexRequest) (*commonpb.Status, error) { func (node *NodeImpl) CreateIndex(request *milvuspb.CreateIndexRequest) (*commonpb.Status, error) {
log.Println("create index for: ", request) log.Println("create index for: ", request)
ctx := context.Background()
cit := &CreateIndexTask{ cit := &CreateIndexTask{
Condition: NewTaskCondition(ctx), Condition: NewTaskCondition(ctx),
CreateIndexRequest: request, CreateIndexRequest: request,
@ -455,8 +469,9 @@ func (node *NodeImpl) CreateIndex(ctx context.Context, request *milvuspb.CreateI
return cit.result, nil return cit.result, nil
} }
func (node *NodeImpl) DescribeIndex(ctx context.Context, request *milvuspb.DescribeIndexRequest) (*milvuspb.DescribeIndexResponse, error) { func (node *NodeImpl) DescribeIndex(request *milvuspb.DescribeIndexRequest) (*milvuspb.DescribeIndexResponse, error) {
log.Println("Describe index for: ", request) log.Println("Describe index for: ", request)
ctx := context.Background()
dit := &DescribeIndexTask{ dit := &DescribeIndexTask{
Condition: NewTaskCondition(ctx), Condition: NewTaskCondition(ctx),
DescribeIndexRequest: request, DescribeIndexRequest: request,
@ -498,8 +513,9 @@ func (node *NodeImpl) DescribeIndex(ctx context.Context, request *milvuspb.Descr
return dit.result, nil return dit.result, nil
} }
func (node *NodeImpl) GetIndexState(ctx context.Context, request *milvuspb.IndexStateRequest) (*milvuspb.IndexStateResponse, error) { func (node *NodeImpl) GetIndexState(request *milvuspb.IndexStateRequest) (*milvuspb.IndexStateResponse, error) {
// log.Println("Describe index progress for: ", request) // log.Println("Describe index progress for: ", request)
ctx := context.Background()
dipt := &GetIndexStateTask{ dipt := &GetIndexStateTask{
Condition: NewTaskCondition(ctx), Condition: NewTaskCondition(ctx),
IndexStateRequest: request, IndexStateRequest: request,
@ -540,7 +556,8 @@ func (node *NodeImpl) GetIndexState(ctx context.Context, request *milvuspb.Index
return dipt.result, nil return dipt.result, nil
} }
func (node *NodeImpl) Insert(ctx context.Context, request *milvuspb.InsertRequest) (*milvuspb.InsertResponse, error) { func (node *NodeImpl) Insert(request *milvuspb.InsertRequest) (*milvuspb.InsertResponse, error) {
ctx := context.Background()
span, ctx := opentracing.StartSpanFromContext(ctx, "insert grpc received") span, ctx := opentracing.StartSpanFromContext(ctx, "insert grpc received")
defer span.Finish() defer span.Finish()
span.SetTag("collection name", request.CollectionName) span.SetTag("collection name", request.CollectionName)
@ -607,7 +624,8 @@ func (node *NodeImpl) Insert(ctx context.Context, request *milvuspb.InsertReques
return it.result, nil return it.result, nil
} }
func (node *NodeImpl) Search(ctx context.Context, request *milvuspb.SearchRequest) (*milvuspb.SearchResults, error) { func (node *NodeImpl) Search(request *milvuspb.SearchRequest) (*milvuspb.SearchResults, error) {
ctx := context.Background()
span, ctx := opentracing.StartSpanFromContext(ctx, "search grpc received") span, ctx := opentracing.StartSpanFromContext(ctx, "search grpc received")
defer span.Finish() defer span.Finish()
span.SetTag("collection name", request.CollectionName) span.SetTag("collection name", request.CollectionName)
@ -664,10 +682,10 @@ func (node *NodeImpl) Search(ctx context.Context, request *milvuspb.SearchReques
return qt.result, nil return qt.result, nil
} }
func (node *NodeImpl) Flush(ctx context.Context, request *milvuspb.FlushRequest) (*commonpb.Status, error) { func (node *NodeImpl) Flush(request *milvuspb.FlushRequest) (*commonpb.Status, error) {
panic("implement me") panic("implement me")
} }
func (node *NodeImpl) GetDdChannel(ctx context.Context, request *commonpb.Empty) (*milvuspb.StringResponse, error) { func (node *NodeImpl) GetDdChannel(request *commonpb.Empty) (*milvuspb.StringResponse, error) {
panic("implement me") panic("implement me")
} }

View File

@ -1,33 +0,0 @@
package proxynode
// This file lists all the parameter proxynode node needs to start,
// not used, just for me to record.
type InitParams struct {
nodeID UniqueID
proxyServiceAddress string
masterAddress string
pulsarAddress string
searchBufSize int
searchChannelNames []string
searchResultBufSize int
searchResultChannelNames []string
subTopicName string
// TODO: this variable dynamic changes, how?
queryNodeNum int
insertBufSize int
insertChannelNames []string
timeTickBufSize int
timeTickChannelNames []string
defaultPartitionName string
maxFieldNum int
maxNameLength int
maxDimension int
}

View File

@ -1,46 +1,93 @@
package proxynode package proxynode
import ( import (
"context"
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb" "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
"github.com/zilliztech/milvus-distributed/internal/proto/indexpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
"github.com/zilliztech/milvus-distributed/internal/proto/proxypb" "github.com/zilliztech/milvus-distributed/internal/proto/proxypb"
"github.com/zilliztech/milvus-distributed/internal/proto/querypb"
) )
type MasterClient interface {
CreateCollection(in *milvuspb.CreateCollectionRequest) (*commonpb.Status, error)
DropCollection(in *milvuspb.DropCollectionRequest) (*commonpb.Status, error)
HasCollection(in *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error)
DescribeCollection(in *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error)
ShowCollections(in *milvuspb.ShowCollectionRequest) (*milvuspb.ShowCollectionResponse, error)
CreatePartition(in *milvuspb.CreatePartitionRequest) (*commonpb.Status, error)
DropPartition(in *milvuspb.DropPartitionRequest) (*commonpb.Status, error)
HasPartition(in *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error)
ShowPartitions(in *milvuspb.ShowPartitionRequest) (*milvuspb.ShowPartitionResponse, error)
CreateIndex(in *milvuspb.CreateIndexRequest) (*commonpb.Status, error)
DescribeIndex(in *milvuspb.DescribeIndexRequest) (*milvuspb.DescribeIndexResponse, error)
}
type IndexServiceClient interface {
GetIndexStates(req *indexpb.IndexStatesRequest) (*indexpb.IndexStatesResponse, error)
GetComponentStates() (*internalpb2.ComponentStates, error)
}
type QueryServiceClient interface {
ShowCollections(req *querypb.ShowCollectionRequest) (*querypb.ShowCollectionResponse, error)
LoadCollection(req *querypb.LoadCollectionRequest) (*commonpb.Status, error)
ReleaseCollection(req *querypb.ReleaseCollectionRequest) (*commonpb.Status, error)
ShowPartitions(req *querypb.ShowPartitionRequest) (*querypb.ShowPartitionResponse, error)
LoadPartitions(req *querypb.LoadPartitionRequest) (*commonpb.Status, error)
ReleasePartitions(req *querypb.ReleasePartitionRequest) (*commonpb.Status, error)
CreateQueryChannel() (*querypb.CreateQueryChannelResponse, error)
GetPartitionStates(req *querypb.PartitionStatesRequest) (*querypb.PartitionStatesResponse, error)
//GetSearchChannelNames() ([]string, error)
//GetSearchResultChannels() ([]string, error)
GetComponentStates() (*internalpb2.ComponentStates, error)
}
type DataServiceClient interface {
AssignSegmentID(req *datapb.AssignSegIDRequest) (*datapb.AssignSegIDResponse, error)
GetInsertChannels(req *datapb.InsertChannelRequest) ([]string, error)
GetComponentStates() (*internalpb2.ComponentStates, error)
}
type ProxyServiceClient interface {
GetTimeTickChannel() (string, error)
RegisterNode(request *proxypb.RegisterNodeRequest) (*proxypb.RegisterNodeResponse, error)
GetComponentStates() (*internalpb2.ComponentStates, error)
}
type ProxyNode interface { type ProxyNode interface {
Init() error Init() error
Start() error Start() error
Stop() error Stop() error
InvalidateCollectionMetaCache(ctx context.Context, request *proxypb.InvalidateCollMetaCacheRequest) (*commonpb.Status, error) InvalidateCollectionMetaCache(request *proxypb.InvalidateCollMetaCacheRequest) (*commonpb.Status, error)
CreateCollection(ctx context.Context, request *milvuspb.CreateCollectionRequest) (*commonpb.Status, error) CreateCollection(request *milvuspb.CreateCollectionRequest) (*commonpb.Status, error)
DropCollection(ctx context.Context, request *milvuspb.DropCollectionRequest) (*commonpb.Status, error) DropCollection(request *milvuspb.DropCollectionRequest) (*commonpb.Status, error)
HasCollection(ctx context.Context, request *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error) HasCollection(request *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error)
LoadCollection(ctx context.Context, request *milvuspb.LoadCollectionRequest) (*commonpb.Status, error) LoadCollection(request *milvuspb.LoadCollectionRequest) (*commonpb.Status, error)
ReleaseCollection(ctx context.Context, request *milvuspb.ReleaseCollectionRequest) (*commonpb.Status, error) ReleaseCollection(request *milvuspb.ReleaseCollectionRequest) (*commonpb.Status, error)
DescribeCollection(ctx context.Context, request *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error) DescribeCollection(request *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error)
GetCollectionStatistics(ctx context.Context, request *milvuspb.CollectionStatsRequest) (*milvuspb.CollectionStatsResponse, error) GetCollectionStatistics(request *milvuspb.CollectionStatsRequest) (*milvuspb.CollectionStatsResponse, error)
ShowCollections(ctx context.Context, request *milvuspb.ShowCollectionRequest) (*milvuspb.ShowCollectionResponse, error) ShowCollections(request *milvuspb.ShowCollectionRequest) (*milvuspb.ShowCollectionResponse, error)
CreatePartition(ctx context.Context, request *milvuspb.CreatePartitionRequest) (*commonpb.Status, error) CreatePartition(request *milvuspb.CreatePartitionRequest) (*commonpb.Status, error)
DropPartition(ctx context.Context, request *milvuspb.DropPartitionRequest) (*commonpb.Status, error) DropPartition(request *milvuspb.DropPartitionRequest) (*commonpb.Status, error)
HasPartition(ctx context.Context, request *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error) HasPartition(request *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error)
LoadPartitions(ctx context.Context, request *milvuspb.LoadPartitonRequest) (*commonpb.Status, error) LoadPartitions(request *milvuspb.LoadPartitonRequest) (*commonpb.Status, error)
ReleasePartitions(ctx context.Context, request *milvuspb.ReleasePartitionRequest) (*commonpb.Status, error) ReleasePartitions(request *milvuspb.ReleasePartitionRequest) (*commonpb.Status, error)
GetPartitionStatistics(ctx context.Context, request *milvuspb.PartitionStatsRequest) (*milvuspb.PartitionStatsResponse, error) GetPartitionStatistics(request *milvuspb.PartitionStatsRequest) (*milvuspb.PartitionStatsResponse, error)
ShowPartitions(ctx context.Context, request *milvuspb.ShowPartitionRequest) (*milvuspb.ShowPartitionResponse, error) ShowPartitions(request *milvuspb.ShowPartitionRequest) (*milvuspb.ShowPartitionResponse, error)
CreateIndex(ctx context.Context, request *milvuspb.CreateIndexRequest) (*commonpb.Status, error) CreateIndex(request *milvuspb.CreateIndexRequest) (*commonpb.Status, error)
DescribeIndex(ctx context.Context, request *milvuspb.DescribeIndexRequest) (*milvuspb.DescribeIndexResponse, error) DescribeIndex(request *milvuspb.DescribeIndexRequest) (*milvuspb.DescribeIndexResponse, error)
GetIndexState(ctx context.Context, request *milvuspb.IndexStateRequest) (*milvuspb.IndexStateResponse, error) GetIndexState(request *milvuspb.IndexStateRequest) (*milvuspb.IndexStateResponse, error)
Insert(ctx context.Context, request *milvuspb.InsertRequest) (*milvuspb.InsertResponse, error) Insert(request *milvuspb.InsertRequest) (*milvuspb.InsertResponse, error)
Search(ctx context.Context, request *milvuspb.SearchRequest) (*milvuspb.SearchResults, error) Search(request *milvuspb.SearchRequest) (*milvuspb.SearchResults, error)
Flush(ctx context.Context, request *milvuspb.FlushRequest) (*commonpb.Status, error) Flush(request *milvuspb.FlushRequest) (*commonpb.Status, error)
GetDdChannel(ctx context.Context, request *commonpb.Empty) (*milvuspb.StringResponse, error) GetDdChannel(request *commonpb.Empty) (*milvuspb.StringResponse, error)
} }

View File

@ -1,25 +1,33 @@
package proxynode package proxynode
import ( import (
"bytes"
"log" "log"
"net"
"os"
"strconv" "strconv"
"strings" "strings"
"time" "time"
"github.com/spf13/cast"
"github.com/spf13/viper"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
"github.com/zilliztech/milvus-distributed/internal/util/paramtable" "github.com/zilliztech/milvus-distributed/internal/util/paramtable"
) )
const (
StartParamsKey = "START_PARAMS"
)
type ParamTable struct { type ParamTable struct {
paramtable.BaseTable paramtable.BaseTable
NetworkPort int NetworkPort int
NetworkAddress string IP string
ProxyServiceAddress string NetworkAddress string
MasterAddress string
PulsarAddress string MasterAddress string
IndexServerAddress string PulsarAddress string
QueryNodeNum int QueryNodeNum int
QueryNodeIDList []UniqueID QueryNodeIDList []UniqueID
ProxyID UniqueID ProxyID UniqueID
@ -45,15 +53,152 @@ type ParamTable struct {
var Params ParamTable var Params ParamTable
func (pt *ParamTable) LoadConfigFromInitParams(initParams *internalpb2.InitParams) error {
pt.ProxyID = initParams.NodeID
config := viper.New()
config.SetConfigType("yaml")
for _, pair := range initParams.StartParams {
if pair.Key == StartParamsKey {
err := config.ReadConfig(bytes.NewBuffer([]byte(pair.Value)))
if err != nil {
return err
}
break
}
}
for _, key := range config.AllKeys() {
val := config.Get(key)
str, err := cast.ToStringE(val)
if err != nil {
switch val := val.(type) {
case []interface{}:
str = str[:0]
for _, v := range val {
ss, err := cast.ToStringE(v)
if err != nil {
log.Panic(err)
}
if len(str) == 0 {
str = ss
} else {
str = str + "," + ss
}
}
default:
log.Panicf("undefine config type, key=%s", key)
}
}
err = pt.Save(key, str)
if err != nil {
panic(err)
}
}
pt.initParams()
//
//pulsarPort := config.GetString(PulsarPort)
//pulsarHost := config.GetString(PulsarHost)
//pt.PulsarAddress = pulsarHost + ":" + pulsarPort
//
//
//queryNodeIDList := config.GetString(QueryNodeIDList)
//pt.QueryNodeIDList = nil
//queryNodeIDs := strings.Split(queryNodeIDList, ",")
//for _, queryNodeID := range queryNodeIDs {
// v, err := strconv.Atoi(queryNodeID)
// if err != nil {
// return err
// }
// pt.QueryNodeIDList = append(pt.QueryNodeIDList, typeutil.UniqueID(v))
//}
//pt.QueryNodeNum = len(pt.QueryNodeIDList)
//
//timeTickInterval := config.GetString(TimeTickInterval)
//interval, err := strconv.Atoi(timeTickInterval)
//if err != nil {
// return err
//}
//pt.TimeTickInterval = time.Duration(interval) * time.Millisecond
//
//subName := config.GetString(SubName)
//pt.ProxySubName = subName
//
//timeTickChannelNames := config.GetString(TimeTickChannelNames)
//pt.ProxyTimeTickChannelNames = []string{timeTickChannelNames}
//
//msgStreamInsertBufSizeStr := config.GetString(MsgStreamInsertBufSize)
//msgStreamInsertBufSize, err := strconv.Atoi(msgStreamInsertBufSizeStr)
//if err != nil {
// return err
//}
//pt.MsgStreamInsertBufSize = int64(msgStreamInsertBufSize)
//
//msgStreamSearchBufSizeStr := config.GetString(MsgStreamSearchBufSize)
//msgStreamSearchBufSize, err := strconv.Atoi(msgStreamSearchBufSizeStr)
//if err != nil {
// return err
//}
//pt.MsgStreamSearchBufSize = int64(msgStreamSearchBufSize)
//
//msgStreamSearchResultBufSizeStr := config.GetString(MsgStreamSearchResultBufSize)
//msgStreamSearchResultBufSize, err := strconv.Atoi(msgStreamSearchResultBufSizeStr)
//if err != nil {
// return err
//}
//pt.MsgStreamSearchResultBufSize = int64(msgStreamSearchResultBufSize)
//
//msgStreamSearchResultPulsarBufSizeStr := config.GetString(MsgStreamSearchResultPulsarBufSize)
//msgStreamSearchResultPulsarBufSize, err := strconv.Atoi(msgStreamSearchResultPulsarBufSizeStr)
//if err != nil {
// return err
//}
//pt.MsgStreamSearchResultPulsarBufSize = int64(msgStreamSearchResultPulsarBufSize)
//
//msgStreamTimeTickBufSizeStr := config.GetString(MsgStreamTimeTickBufSize)
//msgStreamTimeTickBufSize, err := strconv.Atoi(msgStreamTimeTickBufSizeStr)
//if err != nil {
// return err
//}
//pt.MsgStreamTimeTickBufSize = int64(msgStreamTimeTickBufSize)
//
//maxNameLengthStr := config.GetString(MaxNameLength)
//maxNameLength, err := strconv.Atoi(maxNameLengthStr)
//if err != nil {
// return err
//}
//pt.MaxNameLength = int64(maxNameLength)
//
//maxFieldNumStr := config.GetString(MaxFieldNum)
//maxFieldNum, err := strconv.Atoi(maxFieldNumStr)
//if err != nil {
// return err
//}
//pt.MaxFieldNum = int64(maxFieldNum)
//
//maxDimensionStr := config.GetString(MaxDimension)
//maxDimension, err := strconv.Atoi(maxDimensionStr)
//if err != nil {
// return err
//}
//pt.MaxDimension = int64(maxDimension)
//
//defaultPartitionTag := config.GetString(DefaultPartitionTag)
//pt.DefaultPartitionTag = defaultPartitionTag
return nil
}
func (pt *ParamTable) Init() { func (pt *ParamTable) Init() {
pt.BaseTable.Init() pt.BaseTable.Init()
pt.initParams()
}
pt.initNetworkPort() func (pt *ParamTable) initParams() {
pt.initNetworkAddress()
pt.initProxyServiceAddress()
pt.initMasterAddress()
pt.initPulsarAddress() pt.initPulsarAddress()
pt.initIndexServerAddress()
pt.initQueryNodeIDList() pt.initQueryNodeIDList()
pt.initQueryNodeNum() pt.initQueryNodeNum()
pt.initProxyID() pt.initProxyID()
@ -77,72 +222,6 @@ func (pt *ParamTable) Init() {
pt.initDefaultPartitionTag() pt.initDefaultPartitionTag()
} }
func (pt *ParamTable) initNetworkPort() {
pt.NetworkPort = pt.ParseInt("proxyNode.port")
}
func (pt *ParamTable) initNetworkAddress() {
addr, err := pt.Load("proxyNode.address")
if err != nil {
panic(err)
}
hostName, _ := net.LookupHost(addr)
if len(hostName) <= 0 {
if ip := net.ParseIP(addr); ip == nil {
panic("invalid ip proxyNode.address")
}
}
port, err := pt.Load("proxyNode.port")
if err != nil {
panic(err)
}
_, err = strconv.Atoi(port)
if err != nil {
panic(err)
}
pt.NetworkAddress = addr + ":" + port
}
func (pt *ParamTable) initProxyServiceAddress() {
addressFromEnv := os.Getenv("PROXY_SERVICE_ADDRESS")
if len(addressFromEnv) > 0 {
pt.ProxyServiceAddress = addressFromEnv
}
addr, err := pt.Load("proxyService.address")
if err != nil {
panic(err)
}
hostName, _ := net.LookupHost(addr)
if len(hostName) <= 0 {
if ip := net.ParseIP(addr); ip == nil {
panic("invalid ip proxyService.address")
}
}
port, err := pt.Load("proxyService.port")
if err != nil {
panic(err)
}
_, err = strconv.Atoi(port)
if err != nil {
panic(err)
}
pt.ProxyServiceAddress = addr + ":" + port
}
func (pt *ParamTable) initMasterAddress() {
ret, err := pt.Load("_MasterAddress")
if err != nil {
panic(err)
}
pt.MasterAddress = ret
}
func (pt *ParamTable) initPulsarAddress() { func (pt *ParamTable) initPulsarAddress() {
ret, err := pt.Load("_PulsarAddress") ret, err := pt.Load("_PulsarAddress")
if err != nil { if err != nil {
@ -151,31 +230,6 @@ func (pt *ParamTable) initPulsarAddress() {
pt.PulsarAddress = ret pt.PulsarAddress = ret
} }
func (pt *ParamTable) initIndexServerAddress() {
addr, err := pt.Load("indexServer.address")
if err != nil {
panic(err)
}
hostName, _ := net.LookupHost(addr)
if len(hostName) <= 0 {
if ip := net.ParseIP(addr); ip == nil {
panic("invalid ip indexServer.address")
}
}
port, err := pt.Load("indexServer.port")
if err != nil {
panic(err)
}
_, err = strconv.Atoi(port)
if err != nil {
panic(err)
}
pt.IndexServerAddress = addr + ":" + port
}
func (pt *ParamTable) initQueryNodeNum() { func (pt *ParamTable) initQueryNodeNum() {
pt.QueryNodeNum = len(pt.QueryNodeIDList) pt.QueryNodeNum = len(pt.QueryNodeIDList)
} }

View File

@ -8,6 +8,12 @@ import (
"sync" "sync"
"time" "time"
"github.com/zilliztech/milvus-distributed/internal/proto/proxypb"
"github.com/zilliztech/milvus-distributed/internal/util/retry"
"github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms" "github.com/zilliztech/milvus-distributed/internal/msgstream/pulsarms"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2" "github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
@ -32,17 +38,20 @@ type NodeImpl struct {
ip string ip string
port int port int
masterClient MasterClientInterface stateCode internalpb2.StateCode
masterClient MasterClient
indexServiceClient IndexServiceClient indexServiceClient IndexServiceClient
queryServiceClient QueryServiceClient
dataServiceClient DataServiceClient dataServiceClient DataServiceClient
proxyServiceClient ProxyServiceClient
queryServiceClient QueryServiceClient
sched *TaskScheduler sched *TaskScheduler
tick *timeTick tick *timeTick
idAllocator *allocator.IDAllocator idAllocator *allocator.IDAllocator
tsoAllocator *allocator.TimestampAllocator tsoAllocator *allocator.TimestampAllocator
segAssigner *allocator.SegIDAssigner segAssigner *SegIDAssigner
manipulationMsgStream *pulsarms.PulsarMsgStream manipulationMsgStream *pulsarms.PulsarMsgStream
queryMsgStream *pulsarms.PulsarMsgStream queryMsgStream *pulsarms.PulsarMsgStream
@ -55,7 +64,7 @@ type NodeImpl struct {
closeCallbacks []func() closeCallbacks []func()
} }
func CreateProxyNodeImpl(ctx context.Context) (*NodeImpl, error) { func NewProxyNodeImpl(ctx context.Context) (*NodeImpl, error) {
rand.Seed(time.Now().UnixNano()) rand.Seed(time.Now().UnixNano())
ctx1, cancel := context.WithCancel(ctx) ctx1, cancel := context.WithCancel(ctx)
node := &NodeImpl{ node := &NodeImpl{
@ -64,43 +73,111 @@ func CreateProxyNodeImpl(ctx context.Context) (*NodeImpl, error) {
} }
return node, nil return node, nil
}
type Component interface {
GetComponentStates() (*internalpb2.ComponentStates, error)
}
func (node *NodeImpl) waitForServiceReady(service Component, serviceName string) error {
checkFunc := func() error {
resp, err := service.GetComponentStates()
if err != nil {
return err
}
if resp.Status.ErrorCode != commonpb.ErrorCode_SUCCESS {
return errors.New(resp.Status.Reason)
}
if resp.State.StateCode != internalpb2.StateCode_HEALTHY {
return errors.New("")
}
return nil
}
// wait for 10 seconds
err := retry.Retry(10, time.Second, checkFunc)
if err != nil {
errMsg := fmt.Sprintf("ProxyNode wait for %s ready failed", serviceName)
return errors.New(errMsg)
}
return nil
} }
func (node *NodeImpl) Init() error { func (node *NodeImpl) Init() error {
//Params.Init()
var err error // todo wait for proxyservice state changed to Healthy
err = node.masterClient.Init() err := node.waitForServiceReady(node.proxyServiceClient, "ProxyService")
if err != nil {
return err
}
err = node.indexServiceClient.Init()
if err != nil {
return err
}
err = node.queryServiceClient.Init()
if err != nil {
return err
}
err = node.dataServiceClient.Init()
if err != nil { if err != nil {
return err return err
} }
Params.SearchChannelNames, err = node.queryServiceClient.GetSearchChannelNames() request := &proxypb.RegisterNodeRequest{
Address: &commonpb.Address{
Ip: Params.IP,
Port: int64(Params.NetworkPort),
},
}
response, err := node.proxyServiceClient.RegisterNode(request)
if err != nil { if err != nil {
return err return err
} }
Params.SearchResultChannelNames, err = node.queryServiceClient.GetSearchResultChannelNames() if response.Status.ErrorCode != commonpb.ErrorCode_SUCCESS {
return errors.New(response.Status.Reason)
}
err = Params.LoadConfigFromInitParams(response.InitParams)
if err != nil { if err != nil {
return err return err
} }
Params.InsertChannelNames, err = node.dataServiceClient.GetInsertChannelNames()
if err != nil { // wait for dataservice state changed to Healthy
return err if node.dataServiceClient != nil {
err = node.waitForServiceReady(node.dataServiceClient, "DataService")
if err != nil {
return err
}
} }
// wait for queryservice state changed to Healthy
if node.queryServiceClient != nil {
err = node.waitForServiceReady(node.queryServiceClient, "QueryService")
if err != nil {
return err
}
}
// wait for indexservice state changed to Healthy
if node.indexServiceClient != nil {
err = node.waitForServiceReady(node.indexServiceClient, "IndexService")
if err != nil {
return err
}
}
if node.queryServiceClient != nil {
resp, err := node.queryServiceClient.CreateQueryChannel()
if err != nil {
return err
}
if resp.Status.ErrorCode != commonpb.ErrorCode_SUCCESS {
return errors.New(resp.Status.Reason)
}
Params.SearchChannelNames = []string{resp.RequestChannel}
Params.SearchResultChannelNames = []string{resp.ResultChannel}
}
node.UpdateStateCode(internalpb2.StateCode_HEALTHY)
// todo
//Params.InsertChannelNames, err = node.dataServiceClient.GetInsertChannels()
//if err != nil {
// return err
//}
cfg := &config.Configuration{ cfg := &config.Configuration{
ServiceName: "proxynode", ServiceName: "proxynode",
Sampler: &config.SamplerConfig{ Sampler: &config.SamplerConfig{
@ -136,7 +213,7 @@ func (node *NodeImpl) Init() error {
node.tsoAllocator = tsoAllocator node.tsoAllocator = tsoAllocator
node.tsoAllocator.PeerID = Params.ProxyID node.tsoAllocator.PeerID = Params.ProxyID
segAssigner, err := allocator.NewSegIDAssigner(node.ctx, masterAddr, node.lastTick) segAssigner, err := NewSegIDAssigner(node.ctx, node.dataServiceClient, node.lastTick)
if err != nil { if err != nil {
panic(err) panic(err)
} }
@ -162,23 +239,6 @@ func (node *NodeImpl) Init() error {
} }
func (node *NodeImpl) Start() error { func (node *NodeImpl) Start() error {
var err error
err = node.masterClient.Start()
if err != nil {
return err
}
err = node.indexServiceClient.Start()
if err != nil {
return err
}
err = node.queryServiceClient.Start()
if err != nil {
return err
}
err = node.dataServiceClient.Start()
if err != nil {
return err
}
initGlobalMetaCache(node.ctx, node) initGlobalMetaCache(node.ctx, node)
node.manipulationMsgStream.Start() node.manipulationMsgStream.Start()
node.queryMsgStream.Start() node.queryMsgStream.Start()
@ -206,23 +266,6 @@ func (node *NodeImpl) Stop() error {
node.manipulationMsgStream.Close() node.manipulationMsgStream.Close()
node.queryMsgStream.Close() node.queryMsgStream.Close()
node.tick.Close() node.tick.Close()
var err error
err = node.dataServiceClient.Stop()
if err != nil {
return err
}
err = node.queryServiceClient.Stop()
if err != nil {
return err
}
err = node.indexServiceClient.Stop()
if err != nil {
return err
}
err = node.masterClient.Stop()
if err != nil {
return err
}
node.wg.Wait() node.wg.Wait()
@ -253,3 +296,23 @@ func (node *NodeImpl) lastTick() Timestamp {
func (node *NodeImpl) AddCloseCallback(callbacks ...func()) { func (node *NodeImpl) AddCloseCallback(callbacks ...func()) {
node.closeCallbacks = append(node.closeCallbacks, callbacks...) node.closeCallbacks = append(node.closeCallbacks, callbacks...)
} }
func (node *NodeImpl) SetMasterClient(cli MasterClient) {
node.masterClient = cli
}
func (node *NodeImpl) SetIndexServiceClient(cli IndexServiceClient) {
node.indexServiceClient = cli
}
func (node *NodeImpl) SetDataServiceClient(cli DataServiceClient) {
node.dataServiceClient = cli
}
func (node *NodeImpl) SetProxyServiceClient(cli ProxyServiceClient) {
node.proxyServiceClient = cli
}
func (node *NodeImpl) SetQueryServiceClient(cli QueryServiceClient) {
node.queryServiceClient = cli
}

View File

@ -86,7 +86,7 @@ func startMaster(ctx context.Context) {
func startProxy(ctx context.Context) { func startProxy(ctx context.Context) {
svr, err := CreateProxyNodeImpl(ctx) svr, err := NewProxyNodeImpl(ctx)
proxyServer = svr proxyServer = svr
if err != nil { if err != nil {
log.Print("create proxynode failed", zap.Error(err)) log.Print("create proxynode failed", zap.Error(err))
@ -130,7 +130,7 @@ func shutdown() {
} }
func hasCollection(t *testing.T, name string) bool { func hasCollection(t *testing.T, name string) bool {
resp, err := proxyServer.HasCollection(ctx, &milvuspb.HasCollectionRequest{CollectionName: name}) resp, err := proxyServer.HasCollection(&milvuspb.HasCollectionRequest{CollectionName: name})
msg := "Has Collection " + name + " should succeed!" msg := "Has Collection " + name + " should succeed!"
assert.Nil(t, err, msg) assert.Nil(t, err, msg)
return resp.Value return resp.Value
@ -181,7 +181,7 @@ func createCollection(t *testing.T, name string) {
CollectionName: name, CollectionName: name,
Schema: schemaBytes, Schema: schemaBytes,
} }
resp, err := proxyServer.CreateCollection(ctx, req) resp, err := proxyServer.CreateCollection(req)
assert.Nil(t, err) assert.Nil(t, err)
msg := "Create Collection " + name + " should succeed!" msg := "Create Collection " + name + " should succeed!"
assert.Equal(t, resp.ErrorCode, commonpb.ErrorCode_SUCCESS, msg) assert.Equal(t, resp.ErrorCode, commonpb.ErrorCode_SUCCESS, msg)
@ -191,7 +191,7 @@ func dropCollection(t *testing.T, name string) {
req := &milvuspb.DropCollectionRequest{ req := &milvuspb.DropCollectionRequest{
CollectionName: name, CollectionName: name,
} }
resp, err := proxyServer.DropCollection(ctx, req) resp, err := proxyServer.DropCollection(req)
assert.Nil(t, err) assert.Nil(t, err)
msg := "Drop Collection " + name + " should succeed! err :" + resp.Reason msg := "Drop Collection " + name + " should succeed! err :" + resp.Reason
assert.Equal(t, resp.ErrorCode, commonpb.ErrorCode_SUCCESS, msg) assert.Equal(t, resp.ErrorCode, commonpb.ErrorCode_SUCCESS, msg)
@ -210,7 +210,7 @@ func createIndex(t *testing.T, collectionName, fieldName string) {
}, },
} }
resp, err := proxyServer.CreateIndex(ctx, req) resp, err := proxyServer.CreateIndex(req)
assert.Nil(t, err) assert.Nil(t, err)
msg := "Create Index for " + fieldName + " should succeed!" msg := "Create Index for " + fieldName + " should succeed!"
assert.Equal(t, resp.ErrorCode, commonpb.ErrorCode_SUCCESS, msg) assert.Equal(t, resp.ErrorCode, commonpb.ErrorCode_SUCCESS, msg)
@ -262,7 +262,7 @@ func TestProxy_DescribeCollection(t *testing.T) {
createCollection(t, collectionName) createCollection(t, collectionName)
has := hasCollection(t, collectionName) has := hasCollection(t, collectionName)
if has { if has {
resp, err := proxyServer.DescribeCollection(ctx, &milvuspb.DescribeCollectionRequest{CollectionName: collectionName}) resp, err := proxyServer.DescribeCollection(&milvuspb.DescribeCollectionRequest{CollectionName: collectionName})
if err != nil { if err != nil {
t.Error(err) t.Error(err)
} }
@ -288,7 +288,7 @@ func TestProxy_ShowCollections(t *testing.T) {
createCollection(t, collectionName) createCollection(t, collectionName)
has := hasCollection(t, collectionName) has := hasCollection(t, collectionName)
if has { if has {
resp, err := proxyServer.ShowCollections(ctx, &milvuspb.ShowCollectionRequest{}) resp, err := proxyServer.ShowCollections(&milvuspb.ShowCollectionRequest{})
if err != nil { if err != nil {
t.Error(err) t.Error(err)
} }
@ -321,7 +321,7 @@ func TestProxy_Insert(t *testing.T) {
createCollection(t, collectionName) createCollection(t, collectionName)
has := hasCollection(t, collectionName) has := hasCollection(t, collectionName)
if has { if has {
resp, err := proxyServer.Insert(ctx, req) resp, err := proxyServer.Insert(req)
if err != nil { if err != nil {
t.Error(err) t.Error(err)
} }
@ -397,7 +397,7 @@ func TestProxy_Search(t *testing.T) {
if !has { if !has {
createCollection(t, collectionName) createCollection(t, collectionName)
} }
resp, err := proxyServer.Search(ctx, req) resp, err := proxyServer.Search(req)
t.Logf("response of search collection %v: %v", i, resp) t.Logf("response of search collection %v: %v", i, resp)
assert.Nil(t, err) assert.Nil(t, err)
dropCollection(t, collectionName) dropCollection(t, collectionName)
@ -458,7 +458,7 @@ func TestProxy_PartitionGRPC(t *testing.T) {
PartitionName: tag, PartitionName: tag,
} }
stb, err := proxyServer.HasPartition(ctx, preq) stb, err := proxyServer.HasPartition(preq)
assert.Nil(t, err) assert.Nil(t, err)
assert.Equal(t, stb.Status.ErrorCode, commonpb.ErrorCode_SUCCESS) assert.Equal(t, stb.Status.ErrorCode, commonpb.ErrorCode_SUCCESS)
assert.Equal(t, stb.Value, false) assert.Equal(t, stb.Value, false)
@ -467,11 +467,11 @@ func TestProxy_PartitionGRPC(t *testing.T) {
CollectionName: collName, CollectionName: collName,
PartitionName: tag, PartitionName: tag,
} }
st, err := proxyServer.CreatePartition(ctx, cpreq) st, err := proxyServer.CreatePartition(cpreq)
assert.Nil(t, err) assert.Nil(t, err)
assert.Equal(t, st.ErrorCode, commonpb.ErrorCode_SUCCESS) assert.Equal(t, st.ErrorCode, commonpb.ErrorCode_SUCCESS)
stb, err = proxyServer.HasPartition(ctx, preq) stb, err = proxyServer.HasPartition(preq)
assert.Nil(t, err) assert.Nil(t, err)
assert.Equal(t, stb.Status.ErrorCode, commonpb.ErrorCode_SUCCESS) assert.Equal(t, stb.Status.ErrorCode, commonpb.ErrorCode_SUCCESS)
assert.Equal(t, stb.Value, true) assert.Equal(t, stb.Value, true)
@ -480,7 +480,7 @@ func TestProxy_PartitionGRPC(t *testing.T) {
//assert.Nil(t, err) //assert.Nil(t, err)
//assert.Equal(t, std.Status.ErrorCode, commonpb.ErrorCode_SUCCESS) //assert.Equal(t, std.Status.ErrorCode, commonpb.ErrorCode_SUCCESS)
sts, err := proxyServer.ShowPartitions(ctx, &milvuspb.ShowPartitionRequest{CollectionName: collName}) sts, err := proxyServer.ShowPartitions(&milvuspb.ShowPartitionRequest{CollectionName: collName})
assert.Nil(t, err) assert.Nil(t, err)
assert.Equal(t, sts.Status.ErrorCode, commonpb.ErrorCode_SUCCESS) assert.Equal(t, sts.Status.ErrorCode, commonpb.ErrorCode_SUCCESS)
assert.True(t, len(sts.PartitionNames) >= 2) assert.True(t, len(sts.PartitionNames) >= 2)
@ -490,7 +490,7 @@ func TestProxy_PartitionGRPC(t *testing.T) {
CollectionName: collName, CollectionName: collName,
PartitionName: tag, PartitionName: tag,
} }
st, err = proxyServer.DropPartition(ctx, dpreq) st, err = proxyServer.DropPartition(dpreq)
assert.Nil(t, err) assert.Nil(t, err)
assert.Equal(t, st.ErrorCode, commonpb.ErrorCode_SUCCESS) assert.Equal(t, st.ErrorCode, commonpb.ErrorCode_SUCCESS)
}() }()
@ -544,7 +544,7 @@ func TestProxy_DescribeIndex(t *testing.T) {
CollectionName: collName, CollectionName: collName,
FieldName: fieldName, FieldName: fieldName,
} }
resp, err := proxyServer.DescribeIndex(ctx, req) resp, err := proxyServer.DescribeIndex(req)
assert.Nil(t, err) assert.Nil(t, err)
msg := "Describe Index for " + fieldName + "should successed!" msg := "Describe Index for " + fieldName + "should successed!"
assert.Equal(t, resp.Status.ErrorCode, commonpb.ErrorCode_SUCCESS, msg) assert.Equal(t, resp.Status.ErrorCode, commonpb.ErrorCode_SUCCESS, msg)
@ -575,7 +575,7 @@ func TestProxy_GetIndexState(t *testing.T) {
CollectionName: collName, CollectionName: collName,
FieldName: fieldName, FieldName: fieldName,
} }
resp, err := proxyServer.GetIndexState(ctx, req) resp, err := proxyServer.GetIndexState(req)
assert.Nil(t, err) assert.Nil(t, err)
msg := "Describe Index Progress for " + fieldName + "should succeed!" msg := "Describe Index Progress for " + fieldName + "should succeed!"
assert.Equal(t, resp.Status.ErrorCode, commonpb.ErrorCode_SUCCESS, msg) assert.Equal(t, resp.Status.ErrorCode, commonpb.ErrorCode_SUCCESS, msg)

View File

@ -9,7 +9,6 @@ import (
"github.com/zilliztech/milvus-distributed/internal/util/typeutil" "github.com/zilliztech/milvus-distributed/internal/util/typeutil"
"github.com/zilliztech/milvus-distributed/internal/allocator"
"github.com/zilliztech/milvus-distributed/internal/errors" "github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/msgstream" "github.com/zilliztech/milvus-distributed/internal/msgstream"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb" "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
@ -17,7 +16,7 @@ import (
func insertRepackFunc(tsMsgs []msgstream.TsMsg, func insertRepackFunc(tsMsgs []msgstream.TsMsg,
hashKeys [][]int32, hashKeys [][]int32,
segIDAssigner *allocator.SegIDAssigner, segIDAssigner *SegIDAssigner,
together bool) (map[int32]*msgstream.MsgPack, error) { together bool) (map[int32]*msgstream.MsgPack, error) {
result := make(map[int32]*msgstream.MsgPack) result := make(map[int32]*msgstream.MsgPack)

View File

@ -1,4 +1,4 @@
package allocator package proxynode
import ( import (
"container/list" "container/list"
@ -8,6 +8,8 @@ import (
"strconv" "strconv"
"time" "time"
"github.com/zilliztech/milvus-distributed/internal/allocator"
"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"
"github.com/zilliztech/milvus-distributed/internal/proto/datapb" "github.com/zilliztech/milvus-distributed/internal/proto/datapb"
@ -19,6 +21,20 @@ const (
ActiveTimeDuration = 100 //second ActiveTimeDuration = 100 //second
) )
type Allocator = allocator.Allocator
type segRequest struct {
allocator.BaseRequest
count uint32
colName string
partitionName string
collID UniqueID
partitionID UniqueID
segInfo map[UniqueID]uint32
channelID int32
timestamp Timestamp
}
type segInfo struct { type segInfo struct {
segID UniqueID segID UniqueID
count uint32 count uint32
@ -110,30 +126,38 @@ type SegIDAssigner struct {
segReqs []*datapb.SegIDRequest segReqs []*datapb.SegIDRequest
getTickFunc func() Timestamp getTickFunc func() Timestamp
PeerID UniqueID PeerID UniqueID
serviceClient DataServiceClient
countPerRPC uint32
} }
func NewSegIDAssigner(ctx context.Context, masterAddr string, getTickFunc func() Timestamp) (*SegIDAssigner, error) { func NewSegIDAssigner(ctx context.Context, client DataServiceClient, getTickFunc func() Timestamp) (*SegIDAssigner, error) {
ctx1, cancel := context.WithCancel(ctx) ctx1, cancel := context.WithCancel(ctx)
sa := &SegIDAssigner{ sa := &SegIDAssigner{
Allocator: Allocator{reqs: make(chan request, maxConcurrentRequests), Allocator: Allocator{
ctx: ctx1, Ctx: ctx1,
cancel: cancel, CancelFunc: cancel,
masterAddress: masterAddr,
countPerRPC: SegCountPerRPC,
}, },
assignInfos: make(map[string]*list.List), countPerRPC: SegCountPerRPC,
getTickFunc: getTickFunc, serviceClient: client,
assignInfos: make(map[string]*list.List),
getTickFunc: getTickFunc,
} }
sa.tChan = &ticker{ sa.TChan = &allocator.Ticker{
updateInterval: time.Second, UpdateInterval: time.Second,
} }
sa.Allocator.syncFunc = sa.syncSegments sa.Allocator.SyncFunc = sa.syncSegments
sa.Allocator.processFunc = sa.processFunc sa.Allocator.ProcessFunc = sa.processFunc
sa.Allocator.checkSyncFunc = sa.checkSyncFunc sa.Allocator.CheckSyncFunc = sa.checkSyncFunc
sa.Allocator.pickCanDoFunc = sa.pickCanDoFunc sa.Allocator.PickCanDoFunc = sa.pickCanDoFunc
sa.Init()
return sa, nil return sa, nil
} }
func (sa *SegIDAssigner) SetServiceClient(client DataServiceClient) {
sa.serviceClient = client
}
func (sa *SegIDAssigner) collectExpired() { func (sa *SegIDAssigner) collectExpired() {
ts := sa.getTickFunc() ts := sa.getTickFunc()
//now := time.Now() //now := time.Now()
@ -149,12 +173,12 @@ func (sa *SegIDAssigner) collectExpired() {
} }
func (sa *SegIDAssigner) pickCanDoFunc() { func (sa *SegIDAssigner) pickCanDoFunc() {
if sa.toDoReqs == nil { if sa.ToDoReqs == nil {
return return
} }
records := make(map[string]map[string]map[int32]uint32) records := make(map[string]map[string]map[int32]uint32)
newTodoReqs := sa.toDoReqs[0:0] newTodoReqs := sa.ToDoReqs[0:0]
for _, req := range sa.toDoReqs { for _, req := range sa.ToDoReqs {
segRequest := req.(*segRequest) segRequest := req.(*segRequest)
colName := segRequest.colName colName := segRequest.colName
partitionName := segRequest.partitionName partitionName := segRequest.partitionName
@ -185,10 +209,10 @@ func (sa *SegIDAssigner) pickCanDoFunc() {
}) })
newTodoReqs = append(newTodoReqs, req) newTodoReqs = append(newTodoReqs, req)
} else { } else {
sa.canDoReqs = append(sa.canDoReqs, req) sa.CanDoReqs = append(sa.CanDoReqs, req)
} }
} }
sa.toDoReqs = newTodoReqs sa.ToDoReqs = newTodoReqs
} }
func (sa *SegIDAssigner) getAssign(colName, partitionName string, channelID int32) *assignInfo { func (sa *SegIDAssigner) getAssign(colName, partitionName string, channelID int32) *assignInfo {
@ -258,7 +282,7 @@ func (sa *SegIDAssigner) syncSegments() bool {
return true return true
} }
sa.reduceSegReqs() sa.reduceSegReqs()
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) _, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel() defer cancel()
req := &datapb.AssignSegIDRequest{ req := &datapb.AssignSegIDRequest{
NodeID: sa.PeerID, NodeID: sa.PeerID,
@ -267,7 +291,7 @@ func (sa *SegIDAssigner) syncSegments() bool {
} }
sa.segReqs = []*datapb.SegIDRequest{} sa.segReqs = []*datapb.SegIDRequest{}
resp, err := sa.masterClient.AssignSegmentID(ctx, req) resp, err := sa.serviceClient.AssignSegmentID(req)
if err != nil { if err != nil {
log.Println("GRPC AssignSegmentID Failed", resp, err) log.Println("GRPC AssignSegmentID Failed", resp, err)
@ -319,7 +343,7 @@ func (sa *SegIDAssigner) syncSegments() bool {
return success return success
} }
func (sa *SegIDAssigner) processFunc(req request) error { func (sa *SegIDAssigner) processFunc(req allocator.Request) error {
segRequest := req.(*segRequest) segRequest := req.(*segRequest)
assign := sa.getAssign(segRequest.colName, segRequest.partitionName, segRequest.channelID) assign := sa.getAssign(segRequest.colName, segRequest.partitionName, segRequest.channelID)
if assign == nil { if assign == nil {
@ -332,14 +356,14 @@ func (sa *SegIDAssigner) processFunc(req request) error {
func (sa *SegIDAssigner) GetSegmentID(colName, partitionName string, channelID int32, count uint32, ts Timestamp) (map[UniqueID]uint32, error) { func (sa *SegIDAssigner) GetSegmentID(colName, partitionName string, channelID int32, count uint32, ts Timestamp) (map[UniqueID]uint32, error) {
req := &segRequest{ req := &segRequest{
baseRequest: baseRequest{done: make(chan error), valid: false}, BaseRequest: allocator.BaseRequest{Done: make(chan error), Valid: false},
colName: colName, colName: colName,
partitionName: partitionName, partitionName: partitionName,
channelID: channelID, channelID: channelID,
count: count, count: count,
timestamp: ts, timestamp: ts,
} }
sa.reqs <- req sa.Reqs <- req
req.Wait() req.Wait()
if !req.IsValid() { if !req.IsValid() {

View File

@ -188,7 +188,7 @@ func (it *InsertTask) PostExecute() error {
type CreateCollectionTask struct { type CreateCollectionTask struct {
Condition Condition
*milvuspb.CreateCollectionRequest *milvuspb.CreateCollectionRequest
masterClient MasterClientInterface masterClient MasterClient
result *commonpb.Status result *commonpb.Status
ctx context.Context ctx context.Context
schema *schemapb.CollectionSchema schema *schemapb.CollectionSchema
@ -303,7 +303,7 @@ func (cct *CreateCollectionTask) PostExecute() error {
type DropCollectionTask struct { type DropCollectionTask struct {
Condition Condition
*milvuspb.DropCollectionRequest *milvuspb.DropCollectionRequest
masterClient MasterClientInterface masterClient MasterClient
result *commonpb.Status result *commonpb.Status
ctx context.Context ctx context.Context
} }
@ -645,7 +645,7 @@ func (st *SearchTask) PostExecute() error {
type HasCollectionTask struct { type HasCollectionTask struct {
Condition Condition
*milvuspb.HasCollectionRequest *milvuspb.HasCollectionRequest
masterClient MasterClientInterface masterClient MasterClient
result *milvuspb.BoolResponse result *milvuspb.BoolResponse
ctx context.Context ctx context.Context
} }
@ -702,7 +702,7 @@ func (hct *HasCollectionTask) PostExecute() error {
type DescribeCollectionTask struct { type DescribeCollectionTask struct {
Condition Condition
*milvuspb.DescribeCollectionRequest *milvuspb.DescribeCollectionRequest
masterClient MasterClientInterface masterClient MasterClient
result *milvuspb.DescribeCollectionResponse result *milvuspb.DescribeCollectionResponse
ctx context.Context ctx context.Context
} }
@ -763,7 +763,7 @@ func (dct *DescribeCollectionTask) PostExecute() error {
type ShowCollectionsTask struct { type ShowCollectionsTask struct {
Condition Condition
*milvuspb.ShowCollectionRequest *milvuspb.ShowCollectionRequest
masterClient MasterClientInterface masterClient MasterClient
result *milvuspb.ShowCollectionResponse result *milvuspb.ShowCollectionResponse
ctx context.Context ctx context.Context
} }
@ -817,7 +817,7 @@ func (sct *ShowCollectionsTask) PostExecute() error {
type CreatePartitionTask struct { type CreatePartitionTask struct {
Condition Condition
*milvuspb.CreatePartitionRequest *milvuspb.CreatePartitionRequest
masterClient MasterClientInterface masterClient MasterClient
result *commonpb.Status result *commonpb.Status
ctx context.Context ctx context.Context
} }
@ -880,7 +880,7 @@ func (cpt *CreatePartitionTask) PostExecute() error {
type DropPartitionTask struct { type DropPartitionTask struct {
Condition Condition
*milvuspb.DropPartitionRequest *milvuspb.DropPartitionRequest
masterClient MasterClientInterface masterClient MasterClient
result *commonpb.Status result *commonpb.Status
ctx context.Context ctx context.Context
} }
@ -943,7 +943,7 @@ func (dpt *DropPartitionTask) PostExecute() error {
type HasPartitionTask struct { type HasPartitionTask struct {
Condition Condition
*milvuspb.HasPartitionRequest *milvuspb.HasPartitionRequest
masterClient MasterClientInterface masterClient MasterClient
result *milvuspb.BoolResponse result *milvuspb.BoolResponse
ctx context.Context ctx context.Context
} }
@ -1059,7 +1059,7 @@ func (hpt *HasPartitionTask) PostExecute() error {
type ShowPartitionsTask struct { type ShowPartitionsTask struct {
Condition Condition
*milvuspb.ShowPartitionRequest *milvuspb.ShowPartitionRequest
masterClient MasterClientInterface masterClient MasterClient
result *milvuspb.ShowPartitionResponse result *milvuspb.ShowPartitionResponse
ctx context.Context ctx context.Context
} }
@ -1116,7 +1116,7 @@ func (spt *ShowPartitionsTask) PostExecute() error {
type CreateIndexTask struct { type CreateIndexTask struct {
Condition Condition
*milvuspb.CreateIndexRequest *milvuspb.CreateIndexRequest
masterClient MasterClientInterface masterClient MasterClient
result *commonpb.Status result *commonpb.Status
ctx context.Context ctx context.Context
} }
@ -1179,7 +1179,7 @@ func (cit *CreateIndexTask) PostExecute() error {
type DescribeIndexTask struct { type DescribeIndexTask struct {
Condition Condition
*milvuspb.DescribeIndexRequest *milvuspb.DescribeIndexRequest
masterClient MasterClientInterface masterClient MasterClient
result *milvuspb.DescribeIndexResponse result *milvuspb.DescribeIndexResponse
ctx context.Context ctx context.Context
} }

View File

@ -83,6 +83,7 @@ func (s *ServiceImpl) fillNodeInitParams() error {
} }
func (s *ServiceImpl) Init() error { func (s *ServiceImpl) Init() error {
err := s.fillNodeInitParams() err := s.fillNodeInitParams()
if err != nil { if err != nil {
return err return err
@ -102,7 +103,7 @@ func (s *ServiceImpl) Init() error {
ttBarrier := newSoftTimeTickBarrier(s.ctx, nodeTimeTickMsgStream, []UniqueID{0}, 10) ttBarrier := newSoftTimeTickBarrier(s.ctx, nodeTimeTickMsgStream, []UniqueID{0}, 10)
s.tick = newTimeTick(s.ctx, ttBarrier, serviceTimeTickMsgStream) s.tick = newTimeTick(s.ctx, ttBarrier, serviceTimeTickMsgStream)
s.state.State.StateCode = internalpb2.StateCode_HEALTHY s.stateCode = internalpb2.StateCode_HEALTHY
return nil return nil
} }
@ -119,7 +120,24 @@ func (s *ServiceImpl) Stop() error {
} }
func (s *ServiceImpl) GetComponentStates() (*internalpb2.ComponentStates, error) { func (s *ServiceImpl) GetComponentStates() (*internalpb2.ComponentStates, error) {
return s.state, nil stateInfo := &internalpb2.ComponentInfo{
NodeID: UniqueID(0),
Role: "ProxyService",
StateCode: s.stateCode,
}
ret := &internalpb2.ComponentStates{
State: stateInfo,
SubcomponentStates: nil, // todo add subcomponents states
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_SUCCESS,
},
}
return ret, nil
}
func (s *ServiceImpl) UpdateStateCode(code internalpb2.StateCode) {
s.stateCode = code
} }
func (s *ServiceImpl) GetTimeTickChannel() (string, error) { func (s *ServiceImpl) GetTimeTickChannel() (string, error) {

View File

@ -17,8 +17,9 @@ type ServiceImpl struct {
sched *TaskScheduler sched *TaskScheduler
tick TimeTick tick TimeTick
nodeInfos *GlobalNodeInfoTable nodeInfos *GlobalNodeInfoTable
stateCode internalpb2.StateCode
state *internalpb2.ComponentStates //subStates *internalpb2.ComponentStates
dataServiceClient *dataservice.Client dataServiceClient *dataservice.Client
nodeStartParams []*commonpb.KeyValuePair nodeStartParams []*commonpb.KeyValuePair
@ -27,6 +28,23 @@ type ServiceImpl struct {
cancel context.CancelFunc cancel context.CancelFunc
} }
func NewServiceImpl(ctx context.Context) (*ServiceImpl, error) {
rand.Seed(time.Now().UnixNano())
ctx1, cancel := context.WithCancel(ctx)
s := &ServiceImpl{
ctx: ctx1,
cancel: cancel,
}
s.allocator = NewNodeIDAllocator()
s.sched = NewTaskScheduler(ctx1)
s.nodeInfos = NewGlobalNodeInfoTable()
s.stateCode = internalpb2.StateCode_ABNORMAL
return s, nil
}
// deprecated
func CreateProxyService(ctx context.Context) (ProxyService, error) { func CreateProxyService(ctx context.Context) (ProxyService, error) {
rand.Seed(time.Now().UnixNano()) rand.Seed(time.Now().UnixNano())
ctx1, cancel := context.WithCancel(ctx) ctx1, cancel := context.WithCancel(ctx)
@ -38,16 +56,17 @@ func CreateProxyService(ctx context.Context) (ProxyService, error) {
s.allocator = NewNodeIDAllocator() s.allocator = NewNodeIDAllocator()
s.sched = NewTaskScheduler(ctx1) s.sched = NewTaskScheduler(ctx1)
s.nodeInfos = NewGlobalNodeInfoTable() s.nodeInfos = NewGlobalNodeInfoTable()
s.stateCode = internalpb2.StateCode_ABNORMAL
s.state = &internalpb2.ComponentStates{ /*
State: &internalpb2.ComponentInfo{ s.state = &internalpb2.ComponentStates{
NodeID: 0, State: &internalpb2.ComponentInfo{
Role: "proxyservice", NodeID: 0,
StateCode: internalpb2.StateCode_INITIALIZING, Role: "proxyservice",
}, StateCode: internalpb2.StateCode_INITIALIZING,
SubcomponentStates: nil, },
Status: &commonpb.Status{}, SubcomponentStates: nil,
} Status: &commonpb.Status{},
}
*/
return s, nil return s, nil
} }

View File

@ -0,0 +1,32 @@
package funcutil
import (
"context"
"net"
"time"
"github.com/go-basic/ipv4"
)
func CheckGrpcReady(ctx context.Context, targetCh chan error) {
select {
case <-time.After(100 * time.Millisecond):
targetCh <- nil
case <-ctx.Done():
return
}
}
func GetAvailablePort() int {
listener, err := net.Listen("tcp", ":0")
if err != nil {
panic(err)
}
defer listener.Close()
return listener.Addr().(*net.TCPAddr).Port
}
func GetLocalIP() string {
return ipv4.LocalIP()
}

View File

@ -17,10 +17,14 @@ echo $MILVUS_DIR
go test -race -cover "${MILVUS_DIR}/kv/..." -failfast go test -race -cover "${MILVUS_DIR}/kv/..." -failfast
# TODO: remove to distributed # TODO: remove to distributed
#go test -race -cover "${MILVUS_DIR}/proxynode/..." -failfast #go test -race -cover "${MILVUS_DIR}/proxynode/..." -failfast
go test -race -cover "${MILVUS_DIR}/writenode/..." -failfast #go test -race -cover "${MILVUS_DIR}/writenode/..." -failfast
go test -race -cover "${MILVUS_DIR}/datanode/..." -failfast #go test -race -cover "${MILVUS_DIR}/datanode/..." -failfast
go test -race -cover "${MILVUS_DIR}/master/..." -failfast #go test -race -cover "${MILVUS_DIR}/master/..." -failfast
go test -race -cover "${MILVUS_DIR}/indexnode/..." -failfast #go test -race -cover "${MILVUS_DIR}/indexnode/..." -failfast
go test -race -cover "${MILVUS_DIR}/msgstream/..." "${MILVUS_DIR}/querynode/..." "${MILVUS_DIR}/storage" "${MILVUS_DIR}/util/..." -failfast #go test -race -cover "${MILVUS_DIR}/msgstream/..." "${MILVUS_DIR}/querynode/..." "${MILVUS_DIR}/storage" "${MILVUS_DIR}/util/..." -failfast
#go test -race -cover "${MILVUS_DIR}/msgstream/..." "${MILVUS_DIR}/storage" "${MILVUS_DIR}/util/..." -failfast
#go test -race -cover "${MILVUS_DIR}/msgstream/..." "${MILVUS_DIR}/util/..." -failfast
go test -race -cover "${MILVUS_DIR}/msgstream/..." -failfast
go test -race -cover -v "${MILVUS_DIR}/masterservice" "${MILVUS_DIR}/distributed/masterservice" -failfast go test -race -cover -v "${MILVUS_DIR}/masterservice" "${MILVUS_DIR}/distributed/masterservice" -failfast
#go test -race -cover "${MILVUS_DIR}/kv/..." "${MILVUS_DIR}/msgstream/..." "${MILVUS_DIR}/master/..." "${MILVUS_DIR}/querynode/..." -failfast #go test -race -cover "${MILVUS_DIR}/kv/..." "${MILVUS_DIR}/msgstream/..." "${MILVUS_DIR}/master/..." "${MILVUS_DIR}/querynode/..." -failfast