mirror of https://github.com/milvus-io/milvus.git
parent
7957170220
commit
ee0faddb07
2
Makefile
2
Makefile
|
@ -173,7 +173,7 @@ unittest: test-cpp test-go
|
|||
test-go:build-cpp
|
||||
@echo "Running go unittests..."
|
||||
@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
|
||||
@echo "Running cpp unittests..."
|
||||
|
|
|
@ -34,8 +34,8 @@ func main() {
|
|||
cnt := 0
|
||||
|
||||
psc.Params.Init()
|
||||
log.Printf("proxy service address : %s", psc.Params.NetworkAddress())
|
||||
proxyService := psc.NewClient(psc.Params.NetworkAddress())
|
||||
log.Printf("proxy service address : %s", psc.Params.ServiceAddress)
|
||||
proxyService := psc.NewClient(psc.Params.ServiceAddress)
|
||||
|
||||
for cnt = 0; cnt < reTryCnt; cnt++ {
|
||||
pxStates, err := proxyService.GetComponentStates()
|
||||
|
|
|
@ -14,7 +14,7 @@ import (
|
|||
|
||||
func main() {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
svr, err := grpcproxynode.CreateProxyNodeServer()
|
||||
svr, err := grpcproxynode.NewServer()
|
||||
if err != nil {
|
||||
log.Print("create server failed", zap.Error(err))
|
||||
}
|
||||
|
@ -32,14 +32,10 @@ func main() {
|
|||
cancel()
|
||||
}()
|
||||
|
||||
if err := svr.Init(); err != nil {
|
||||
if err := svr.Run(); err != nil {
|
||||
log.Fatal("Init server failed", zap.Error(err))
|
||||
}
|
||||
|
||||
if err := svr.Start(); err != nil {
|
||||
log.Fatal("run server failed", zap.Error(err))
|
||||
}
|
||||
|
||||
<-ctx.Done()
|
||||
log.Print("Got signal to exit", zap.String("signal", sig.String()))
|
||||
|
||||
|
|
|
@ -14,7 +14,7 @@ import (
|
|||
|
||||
func main() {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
svr, err := grpcproxyservice.CreateProxyServiceServer()
|
||||
svr, err := grpcproxyservice.NewServer()
|
||||
if err != nil {
|
||||
log.Print("create server failed", zap.Error(err))
|
||||
}
|
||||
|
@ -32,14 +32,10 @@ func main() {
|
|||
cancel()
|
||||
}()
|
||||
|
||||
if err := svr.Init(); err != nil {
|
||||
if err := svr.Run(); err != nil {
|
||||
log.Fatal("init server failed", zap.Error(err))
|
||||
}
|
||||
|
||||
if err := svr.Start(); err != nil {
|
||||
log.Fatal("run server failed", zap.Error(err))
|
||||
}
|
||||
|
||||
<-ctx.Done()
|
||||
log.Print("Got signal to exit", zap.String("signal", sig.String()))
|
||||
|
||||
|
|
|
@ -65,7 +65,7 @@ func InitProxy(wg *sync.WaitGroup) {
|
|||
//proxynode.Init()
|
||||
//fmt.Println("ProxyID is", proxynode.Params.ProxyID())
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
svr, err := proxynode.CreateProxyNodeImpl(ctx)
|
||||
svr, err := proxynode.NewProxyNodeImpl(ctx)
|
||||
if err != nil {
|
||||
log.Print("create server failed", zap.Error(err))
|
||||
}
|
||||
|
|
|
@ -3,192 +3,151 @@ package allocator
|
|||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"log"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
const (
|
||||
maxConcurrentRequests = 10000
|
||||
)
|
||||
|
||||
type request interface {
|
||||
type Request interface {
|
||||
Wait()
|
||||
Notify(error)
|
||||
IsValid() bool
|
||||
}
|
||||
|
||||
type baseRequest struct {
|
||||
done chan error
|
||||
valid bool
|
||||
type BaseRequest struct {
|
||||
Done chan error
|
||||
Valid bool
|
||||
}
|
||||
|
||||
func (req *baseRequest) Wait() {
|
||||
err := <-req.done
|
||||
req.valid = err == nil
|
||||
func (req *BaseRequest) Wait() {
|
||||
err := <-req.Done
|
||||
req.Valid = err == nil
|
||||
}
|
||||
|
||||
func (req *baseRequest) IsValid() bool {
|
||||
return req.valid
|
||||
func (req *BaseRequest) IsValid() bool {
|
||||
return req.Valid
|
||||
}
|
||||
|
||||
func (req *baseRequest) Notify(err error) {
|
||||
req.done <- err
|
||||
func (req *BaseRequest) Notify(err error) {
|
||||
req.Done <- err
|
||||
}
|
||||
|
||||
type idRequest struct {
|
||||
baseRequest
|
||||
type IDRequest struct {
|
||||
BaseRequest
|
||||
id UniqueID
|
||||
count uint32
|
||||
}
|
||||
|
||||
type tsoRequest struct {
|
||||
baseRequest
|
||||
type TSORequest struct {
|
||||
BaseRequest
|
||||
timestamp Timestamp
|
||||
count uint32
|
||||
}
|
||||
|
||||
type segRequest struct {
|
||||
baseRequest
|
||||
count uint32
|
||||
colName string
|
||||
partitionName string
|
||||
collID UniqueID
|
||||
partitionID UniqueID
|
||||
segInfo map[UniqueID]uint32
|
||||
channelID int32
|
||||
timestamp Timestamp
|
||||
type SyncRequest struct {
|
||||
BaseRequest
|
||||
}
|
||||
|
||||
type syncRequest struct {
|
||||
baseRequest
|
||||
}
|
||||
|
||||
type tickerChan interface {
|
||||
type TickerChan interface {
|
||||
Chan() <-chan time.Time
|
||||
Close()
|
||||
Init()
|
||||
Reset()
|
||||
}
|
||||
|
||||
type emptyTicker struct {
|
||||
type EmptyTicker struct {
|
||||
tChan <-chan time.Time
|
||||
}
|
||||
|
||||
func (t *emptyTicker) Chan() <-chan time.Time {
|
||||
func (t *EmptyTicker) Chan() <-chan time.Time {
|
||||
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
|
||||
updateInterval time.Duration //
|
||||
UpdateInterval time.Duration //
|
||||
}
|
||||
|
||||
func (t *ticker) Init() {
|
||||
t.ticker = time.NewTicker(t.updateInterval)
|
||||
func (t *Ticker) Init() {
|
||||
t.ticker = time.NewTicker(t.UpdateInterval)
|
||||
}
|
||||
|
||||
func (t *ticker) Reset() {
|
||||
t.ticker.Reset(t.updateInterval)
|
||||
func (t *Ticker) Reset() {
|
||||
t.ticker.Reset(t.UpdateInterval)
|
||||
}
|
||||
|
||||
func (t *ticker) Close() {
|
||||
func (t *Ticker) Close() {
|
||||
t.ticker.Stop()
|
||||
}
|
||||
|
||||
func (t *ticker) Chan() <-chan time.Time {
|
||||
func (t *Ticker) Chan() <-chan time.Time {
|
||||
return t.ticker.C
|
||||
}
|
||||
|
||||
type Allocator struct {
|
||||
reqs chan request
|
||||
Ctx context.Context
|
||||
CancelFunc context.CancelFunc
|
||||
|
||||
wg sync.WaitGroup
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
wg sync.WaitGroup
|
||||
|
||||
masterAddress string
|
||||
masterConn *grpc.ClientConn
|
||||
masterClient masterpb.MasterServiceClient
|
||||
countPerRPC uint32
|
||||
Reqs chan Request
|
||||
ToDoReqs []Request
|
||||
CanDoReqs []Request
|
||||
SyncReqs []Request
|
||||
|
||||
toDoReqs []request
|
||||
canDoReqs []request
|
||||
syncReqs []request
|
||||
TChan TickerChan
|
||||
ForceSyncChan chan Request
|
||||
|
||||
tChan tickerChan
|
||||
forceSyncChan chan request
|
||||
SyncFunc func() bool
|
||||
ProcessFunc func(req Request) error
|
||||
|
||||
syncFunc func() bool
|
||||
processFunc func(req request) error
|
||||
|
||||
checkSyncFunc func(timeout bool) bool
|
||||
pickCanDoFunc func()
|
||||
CheckSyncFunc func(timeout bool) bool
|
||||
PickCanDoFunc func()
|
||||
}
|
||||
|
||||
func (ta *Allocator) Start() error {
|
||||
connectMasterFn := func() error {
|
||||
return ta.connectMaster()
|
||||
}
|
||||
err := Retry(10, time.Millisecond*200, connectMasterFn)
|
||||
if err != nil {
|
||||
panic("connect to master failed")
|
||||
}
|
||||
ta.tChan.Init()
|
||||
ta.TChan.Init()
|
||||
ta.wg.Add(1)
|
||||
go ta.mainLoop()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ta *Allocator) 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 *Allocator) init() {
|
||||
ta.forceSyncChan = make(chan request, maxConcurrentRequests)
|
||||
func (ta *Allocator) Init() {
|
||||
ta.ForceSyncChan = make(chan Request, maxConcurrentRequests)
|
||||
ta.Reqs = make(chan Request, maxConcurrentRequests)
|
||||
}
|
||||
|
||||
func (ta *Allocator) mainLoop() {
|
||||
defer ta.wg.Done()
|
||||
|
||||
loopCtx, loopCancel := context.WithCancel(ta.ctx)
|
||||
loopCtx, loopCancel := context.WithCancel(ta.Ctx)
|
||||
defer loopCancel()
|
||||
|
||||
for {
|
||||
select {
|
||||
|
||||
case first := <-ta.forceSyncChan:
|
||||
ta.syncReqs = append(ta.syncReqs, first)
|
||||
pending := len(ta.forceSyncChan)
|
||||
case first := <-ta.ForceSyncChan:
|
||||
ta.SyncReqs = append(ta.SyncReqs, first)
|
||||
pending := len(ta.ForceSyncChan)
|
||||
for i := 0; i < pending; i++ {
|
||||
ta.syncReqs = append(ta.syncReqs, <-ta.forceSyncChan)
|
||||
ta.SyncReqs = append(ta.SyncReqs, <-ta.ForceSyncChan)
|
||||
}
|
||||
ta.sync(true)
|
||||
ta.finishSyncRequest()
|
||||
|
||||
case <-ta.tChan.Chan():
|
||||
case <-ta.TChan.Chan():
|
||||
ta.pickCanDo()
|
||||
ta.finishRequest()
|
||||
if ta.sync(true) {
|
||||
|
@ -197,11 +156,11 @@ func (ta *Allocator) mainLoop() {
|
|||
}
|
||||
ta.failRemainRequest()
|
||||
|
||||
case first := <-ta.reqs:
|
||||
ta.toDoReqs = append(ta.toDoReqs, first)
|
||||
pending := len(ta.reqs)
|
||||
case first := <-ta.Reqs:
|
||||
ta.ToDoReqs = append(ta.ToDoReqs, first)
|
||||
pending := len(ta.Reqs)
|
||||
for i := 0; i < pending; i++ {
|
||||
ta.toDoReqs = append(ta.toDoReqs, <-ta.reqs)
|
||||
ta.ToDoReqs = append(ta.ToDoReqs, <-ta.Reqs)
|
||||
}
|
||||
ta.pickCanDo()
|
||||
ta.finishRequest()
|
||||
|
@ -219,78 +178,78 @@ func (ta *Allocator) mainLoop() {
|
|||
}
|
||||
|
||||
func (ta *Allocator) pickCanDo() {
|
||||
if ta.pickCanDoFunc == nil {
|
||||
if ta.PickCanDoFunc == nil {
|
||||
return
|
||||
}
|
||||
ta.pickCanDoFunc()
|
||||
ta.PickCanDoFunc()
|
||||
}
|
||||
|
||||
func (ta *Allocator) sync(timeout bool) bool {
|
||||
if ta.syncFunc == nil || ta.checkSyncFunc == nil {
|
||||
ta.canDoReqs = ta.toDoReqs
|
||||
ta.toDoReqs = ta.toDoReqs[0:0]
|
||||
if ta.SyncFunc == nil || ta.CheckSyncFunc == nil {
|
||||
ta.CanDoReqs = ta.ToDoReqs
|
||||
ta.ToDoReqs = ta.ToDoReqs[0:0]
|
||||
return true
|
||||
}
|
||||
if !timeout && len(ta.toDoReqs) == 0 {
|
||||
if !timeout && len(ta.ToDoReqs) == 0 {
|
||||
return false
|
||||
}
|
||||
if !ta.checkSyncFunc(timeout) {
|
||||
if !ta.CheckSyncFunc(timeout) {
|
||||
return false
|
||||
}
|
||||
|
||||
ret := ta.syncFunc()
|
||||
ret := ta.SyncFunc()
|
||||
|
||||
if !timeout {
|
||||
ta.tChan.Reset()
|
||||
ta.TChan.Reset()
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func (ta *Allocator) finishSyncRequest() {
|
||||
for _, req := range ta.syncReqs {
|
||||
for _, req := range ta.SyncReqs {
|
||||
if req != nil {
|
||||
req.Notify(nil)
|
||||
}
|
||||
}
|
||||
ta.syncReqs = ta.syncReqs[0:0]
|
||||
ta.SyncReqs = ta.SyncReqs[0:0]
|
||||
}
|
||||
|
||||
func (ta *Allocator) failRemainRequest() {
|
||||
for _, req := range ta.toDoReqs {
|
||||
for _, req := range ta.ToDoReqs {
|
||||
if req != nil {
|
||||
req.Notify(errors.New("failed: unexpected error"))
|
||||
}
|
||||
}
|
||||
ta.toDoReqs = []request{}
|
||||
ta.ToDoReqs = []Request{}
|
||||
}
|
||||
|
||||
func (ta *Allocator) finishRequest() {
|
||||
for _, req := range ta.canDoReqs {
|
||||
for _, req := range ta.CanDoReqs {
|
||||
if req != nil {
|
||||
err := ta.processFunc(req)
|
||||
err := ta.ProcessFunc(req)
|
||||
req.Notify(err)
|
||||
}
|
||||
}
|
||||
ta.canDoReqs = []request{}
|
||||
ta.CanDoReqs = []Request{}
|
||||
}
|
||||
|
||||
func (ta *Allocator) revokeRequest(err error) {
|
||||
n := len(ta.reqs)
|
||||
n := len(ta.Reqs)
|
||||
for i := 0; i < n; i++ {
|
||||
req := <-ta.reqs
|
||||
req := <-ta.Reqs
|
||||
req.Notify(err)
|
||||
}
|
||||
}
|
||||
|
||||
func (ta *Allocator) Close() {
|
||||
ta.cancel()
|
||||
ta.CancelFunc()
|
||||
ta.wg.Wait()
|
||||
ta.tChan.Close()
|
||||
ta.TChan.Close()
|
||||
ta.revokeRequest(errors.New("closing"))
|
||||
}
|
||||
|
||||
func (ta *Allocator) CleanCache() {
|
||||
req := &syncRequest{baseRequest: baseRequest{done: make(chan error), valid: false}}
|
||||
ta.forceSyncChan <- req
|
||||
req := &SyncRequest{BaseRequest: BaseRequest{Done: make(chan error), Valid: false}}
|
||||
ta.ForceSyncChan <- req
|
||||
req.Wait()
|
||||
}
|
||||
|
|
|
@ -5,6 +5,9 @@ import (
|
|||
"log"
|
||||
"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/masterpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||
|
@ -19,6 +22,12 @@ type UniqueID = typeutil.UniqueID
|
|||
type IDAllocator struct {
|
||||
Allocator
|
||||
|
||||
masterAddress string
|
||||
masterConn *grpc.ClientConn
|
||||
masterClient masterpb.MasterServiceClient
|
||||
|
||||
countPerRPC uint32
|
||||
|
||||
idStart UniqueID
|
||||
idEnd UniqueID
|
||||
|
||||
|
@ -29,22 +38,48 @@ func NewIDAllocator(ctx context.Context, masterAddr string) (*IDAllocator, error
|
|||
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
a := &IDAllocator{
|
||||
Allocator: Allocator{reqs: make(chan request, maxConcurrentRequests),
|
||||
ctx: ctx1,
|
||||
cancel: cancel,
|
||||
masterAddress: masterAddr,
|
||||
countPerRPC: IDCountPerRPC,
|
||||
Allocator: Allocator{
|
||||
Ctx: ctx1,
|
||||
CancelFunc: cancel,
|
||||
},
|
||||
countPerRPC: IDCountPerRPC,
|
||||
masterAddress: masterAddr,
|
||||
}
|
||||
a.tChan = &emptyTicker{}
|
||||
a.Allocator.syncFunc = a.syncID
|
||||
a.Allocator.processFunc = a.processFunc
|
||||
a.Allocator.checkSyncFunc = a.checkSyncFunc
|
||||
a.Allocator.pickCanDoFunc = a.pickCanDoFunc
|
||||
a.init()
|
||||
a.TChan = &EmptyTicker{}
|
||||
a.Allocator.SyncFunc = a.syncID
|
||||
a.Allocator.ProcessFunc = a.processFunc
|
||||
a.Allocator.CheckSyncFunc = a.checkSyncFunc
|
||||
a.Allocator.PickCanDoFunc = a.pickCanDoFunc
|
||||
a.Init()
|
||||
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 {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
|
||||
req := &masterpb.IDRequest{
|
||||
|
@ -69,28 +104,28 @@ func (ia *IDAllocator) syncID() bool {
|
|||
}
|
||||
|
||||
func (ia *IDAllocator) checkSyncFunc(timeout bool) bool {
|
||||
return timeout || len(ia.toDoReqs) > 0
|
||||
return timeout || len(ia.ToDoReqs) > 0
|
||||
}
|
||||
|
||||
func (ia *IDAllocator) pickCanDoFunc() {
|
||||
total := uint32(ia.idEnd - ia.idStart)
|
||||
need := uint32(0)
|
||||
idx := 0
|
||||
for _, req := range ia.toDoReqs {
|
||||
iReq := req.(*idRequest)
|
||||
for _, req := range ia.ToDoReqs {
|
||||
iReq := req.(*IDRequest)
|
||||
need += iReq.count
|
||||
if need <= total {
|
||||
ia.canDoReqs = append(ia.canDoReqs, req)
|
||||
ia.CanDoReqs = append(ia.CanDoReqs, req)
|
||||
idx++
|
||||
} else {
|
||||
break
|
||||
}
|
||||
}
|
||||
ia.toDoReqs = ia.toDoReqs[idx:]
|
||||
ia.ToDoReqs = ia.ToDoReqs[idx:]
|
||||
}
|
||||
|
||||
func (ia *IDAllocator) processFunc(req request) error {
|
||||
idRequest := req.(*idRequest)
|
||||
func (ia *IDAllocator) processFunc(req Request) error {
|
||||
idRequest := req.(*IDRequest)
|
||||
idRequest.id = ia.idStart
|
||||
ia.idStart++
|
||||
return nil
|
||||
|
@ -105,10 +140,10 @@ func (ia *IDAllocator) AllocOne() (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
|
||||
ia.reqs <- req
|
||||
ia.Reqs <- req
|
||||
req.Wait()
|
||||
|
||||
if !req.IsValid() {
|
||||
|
|
|
@ -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}
|
||||
}
|
|
@ -3,6 +3,9 @@ package allocator
|
|||
import (
|
||||
"context"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/retry"
|
||||
"google.golang.org/grpc"
|
||||
|
||||
"log"
|
||||
"time"
|
||||
|
||||
|
@ -19,6 +22,12 @@ const (
|
|||
|
||||
type TimestampAllocator struct {
|
||||
Allocator
|
||||
|
||||
masterAddress string
|
||||
masterConn *grpc.ClientConn
|
||||
masterClient masterpb.MasterServiceClient
|
||||
|
||||
countPerRPC uint32
|
||||
lastTsBegin Timestamp
|
||||
lastTsEnd Timestamp
|
||||
PeerID UniqueID
|
||||
|
@ -27,42 +36,69 @@ type TimestampAllocator struct {
|
|||
func NewTimestampAllocator(ctx context.Context, masterAddr string) (*TimestampAllocator, error) {
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
a := &TimestampAllocator{
|
||||
Allocator: Allocator{reqs: make(chan request, maxConcurrentRequests),
|
||||
ctx: ctx1,
|
||||
cancel: cancel,
|
||||
masterAddress: masterAddr,
|
||||
countPerRPC: tsCountPerRPC,
|
||||
Allocator: Allocator{
|
||||
Ctx: ctx1,
|
||||
CancelFunc: cancel,
|
||||
},
|
||||
masterAddress: masterAddr,
|
||||
countPerRPC: tsCountPerRPC,
|
||||
}
|
||||
a.tChan = &ticker{
|
||||
updateInterval: time.Second,
|
||||
a.TChan = &Ticker{
|
||||
UpdateInterval: time.Second,
|
||||
}
|
||||
a.Allocator.syncFunc = a.syncTs
|
||||
a.Allocator.processFunc = a.processFunc
|
||||
a.Allocator.checkSyncFunc = a.checkSyncFunc
|
||||
a.Allocator.pickCanDoFunc = a.pickCanDoFunc
|
||||
a.Allocator.SyncFunc = a.syncTs
|
||||
a.Allocator.ProcessFunc = a.processFunc
|
||||
a.Allocator.CheckSyncFunc = a.checkSyncFunc
|
||||
a.Allocator.PickCanDoFunc = a.pickCanDoFunc
|
||||
a.Init()
|
||||
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 {
|
||||
return timeout || len(ta.toDoReqs) > 0
|
||||
return timeout || len(ta.ToDoReqs) > 0
|
||||
}
|
||||
|
||||
func (ta *TimestampAllocator) pickCanDoFunc() {
|
||||
total := uint32(ta.lastTsEnd - ta.lastTsBegin)
|
||||
need := uint32(0)
|
||||
idx := 0
|
||||
for _, req := range ta.toDoReqs {
|
||||
tReq := req.(*tsoRequest)
|
||||
for _, req := range ta.ToDoReqs {
|
||||
tReq := req.(*TSORequest)
|
||||
need += tReq.count
|
||||
if need <= total {
|
||||
ta.canDoReqs = append(ta.canDoReqs, req)
|
||||
ta.CanDoReqs = append(ta.CanDoReqs, req)
|
||||
idx++
|
||||
} else {
|
||||
break
|
||||
}
|
||||
}
|
||||
ta.toDoReqs = ta.toDoReqs[idx:]
|
||||
ta.ToDoReqs = ta.ToDoReqs[idx:]
|
||||
}
|
||||
|
||||
func (ta *TimestampAllocator) syncTs() bool {
|
||||
|
@ -88,8 +124,8 @@ func (ta *TimestampAllocator) syncTs() bool {
|
|||
return true
|
||||
}
|
||||
|
||||
func (ta *TimestampAllocator) processFunc(req request) error {
|
||||
tsoRequest := req.(*tsoRequest)
|
||||
func (ta *TimestampAllocator) processFunc(req Request) error {
|
||||
tsoRequest := req.(*TSORequest)
|
||||
tsoRequest.timestamp = ta.lastTsBegin
|
||||
ta.lastTsBegin++
|
||||
return nil
|
||||
|
@ -104,11 +140,11 @@ func (ta *TimestampAllocator) AllocOne() (Timestamp, error) {
|
|||
}
|
||||
|
||||
func (ta *TimestampAllocator) Alloc(count uint32) ([]Timestamp, error) {
|
||||
req := &tsoRequest{
|
||||
baseRequest: baseRequest{done: make(chan error), valid: false},
|
||||
req := &TSORequest{
|
||||
BaseRequest: BaseRequest{Done: make(chan error), Valid: false},
|
||||
}
|
||||
req.count = count
|
||||
ta.reqs <- req
|
||||
ta.Reqs <- req
|
||||
req.Wait()
|
||||
|
||||
if !req.IsValid() {
|
||||
|
|
|
@ -49,6 +49,7 @@ func (c *GrpcClient) Init() error {
|
|||
c.grpcClient = masterpb.NewMasterServiceClient(c.conn)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *GrpcClient) Start() error {
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -9,7 +9,6 @@ import (
|
|||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||
cms "github.com/zilliztech/milvus-distributed/internal/masterservice"
|
||||
"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/masterpb"
|
||||
"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) {
|
||||
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")
|
||||
}
|
||||
|
|
|
@ -2,37 +2,45 @@ package grpcproxynode
|
|||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
"time"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/proxypb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/retry"
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
type Client struct {
|
||||
client proxypb.ProxyNodeServiceClient
|
||||
address string
|
||||
ctx context.Context
|
||||
grpcClient proxypb.ProxyNodeServiceClient
|
||||
address string
|
||||
ctx context.Context
|
||||
}
|
||||
|
||||
func (c *Client) tryConnect() error {
|
||||
if c.client != nil {
|
||||
func (c *Client) Init() error {
|
||||
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
|
||||
}
|
||||
conn, err := grpc.DialContext(c.ctx, c.address, grpc.WithInsecure(), grpc.WithBlock())
|
||||
err := retry.Retry(10, time.Millisecond*200, connectGrpcFunc)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
c.client = proxypb.NewProxyNodeServiceClient(conn)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *Client) Start() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *Client) Stop() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *Client) InvalidateCollectionMetaCache(request *proxypb.InvalidateCollMetaCacheRequest) error {
|
||||
var err error
|
||||
err = c.tryConnect()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
_, err = c.client.InvalidateCollectionMetaCache(c.ctx, request)
|
||||
_, err := c.grpcClient.InvalidateCollectionMetaCache(c.ctx, request)
|
||||
return err
|
||||
}
|
||||
|
||||
|
|
|
@ -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"
|
||||
)
|
|
@ -2,6 +2,7 @@ package grpcproxynode
|
|||
|
||||
import (
|
||||
"net"
|
||||
"os"
|
||||
"strconv"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/paramtable"
|
||||
|
@ -11,14 +12,68 @@ type ParamTable struct {
|
|||
paramtable.BaseTable
|
||||
|
||||
ProxyServiceAddress string
|
||||
ProxyServicePort int
|
||||
|
||||
IndexServerAddress string
|
||||
MasterAddress string
|
||||
|
||||
DataServiceAddress string
|
||||
QueryServiceAddress string
|
||||
|
||||
IP string
|
||||
Port int
|
||||
Address string
|
||||
}
|
||||
|
||||
var Params ParamTable
|
||||
|
||||
func (pt *ParamTable) 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.initMasterAddress()
|
||||
pt.initIndexServerAddress()
|
||||
pt.initDataServiceAddress()
|
||||
pt.initQueryServiceAddress()
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initPoxyServicePort() {
|
||||
pt.ProxyServicePort = pt.ParseInt("proxyService.port")
|
||||
}
|
||||
|
||||
func (pt *ParamTable) initProxyServiceAddress() {
|
||||
|
@ -44,3 +99,72 @@ func (pt *ParamTable) initProxyServiceAddress() {
|
|||
}
|
||||
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
|
||||
}
|
||||
|
|
|
@ -1,239 +1,196 @@
|
|||
package grpcproxynode
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"log"
|
||||
"net"
|
||||
"os"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||
|
||||
"github.com/spf13/viper"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/funcutil"
|
||||
|
||||
"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"
|
||||
|
||||
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/milvuspb"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/proxynode"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/proxypb"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proxynode"
|
||||
)
|
||||
|
||||
type Server struct {
|
||||
ctx context.Context
|
||||
wg sync.WaitGroup
|
||||
impl proxynode.ProxyNode
|
||||
grpcServer *grpc.Server
|
||||
ip string
|
||||
port int
|
||||
proxyServiceAddress string
|
||||
proxyServiceClient *grpcproxyservice.Client
|
||||
ctx context.Context
|
||||
wg sync.WaitGroup
|
||||
impl *proxynode.NodeImpl
|
||||
grpcServer *grpc.Server
|
||||
|
||||
grpcErrChan chan error
|
||||
|
||||
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) {
|
||||
return &Server{}, nil
|
||||
}
|
||||
func NewServer() (*Server, error) {
|
||||
|
||||
func (s *Server) loadConfigFromInitParams(initParams *internalpb2.InitParams) error {
|
||||
proxynode.Params.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
|
||||
}
|
||||
server := &Server{
|
||||
ctx: context.Background(),
|
||||
grpcErrChan: make(chan error),
|
||||
}
|
||||
|
||||
masterPort := config.GetString(MasterPort)
|
||||
masterHost := config.GetString(MasterHost)
|
||||
proxynode.Params.MasterAddress = masterHost + ":" + masterPort
|
||||
var err error
|
||||
server.impl, err = proxynode.NewProxyNodeImpl(server.ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return server, err
|
||||
}
|
||||
|
||||
pulsarPort := config.GetString(PulsarPort)
|
||||
pulsarHost := config.GetString(PulsarHost)
|
||||
proxynode.Params.PulsarAddress = pulsarHost + ":" + pulsarPort
|
||||
func (s *Server) startGrpcLoop(grpcPort int) {
|
||||
|
||||
indexServerPort := config.GetString(IndexServerPort)
|
||||
indexServerHost := config.GetString(IndexServerHost)
|
||||
proxynode.Params.IndexServerAddress = indexServerHost + ":" + indexServerPort
|
||||
defer s.wg.Done()
|
||||
|
||||
queryNodeIDList := config.GetString(QueryNodeIDList)
|
||||
proxynode.Params.QueryNodeIDList = nil
|
||||
queryNodeIDs := strings.Split(queryNodeIDList, ",")
|
||||
for _, queryNodeID := range queryNodeIDs {
|
||||
v, err := strconv.Atoi(queryNodeID)
|
||||
log.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.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 {
|
||||
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)
|
||||
interval, err := strconv.Atoi(timeTickInterval)
|
||||
s.wg.Add(1)
|
||||
s.startGrpcLoop(Params.Port)
|
||||
// wait for grpc server loop start
|
||||
err = <-s.grpcErrChan
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
proxynode.Params.TimeTickInterval = time.Duration(interval) * time.Millisecond
|
||||
|
||||
subName := config.GetString(SubName)
|
||||
proxynode.Params.ProxySubName = subName
|
||||
|
||||
timeTickChannelNames := config.GetString(TimeTickChannelNames)
|
||||
proxynode.Params.ProxyTimeTickChannelNames = []string{timeTickChannelNames}
|
||||
|
||||
msgStreamInsertBufSizeStr := config.GetString(MsgStreamInsertBufSize)
|
||||
msgStreamInsertBufSize, err := strconv.Atoi(msgStreamInsertBufSizeStr)
|
||||
s.proxyServiceClient = grpcproxyservice.NewClient(Params.ProxyServiceAddress)
|
||||
err = s.proxyServiceClient.Init()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
proxynode.Params.MsgStreamInsertBufSize = int64(msgStreamInsertBufSize)
|
||||
s.impl.SetProxyServiceClient(s.proxyServiceClient)
|
||||
|
||||
msgStreamSearchBufSizeStr := config.GetString(MsgStreamSearchBufSize)
|
||||
msgStreamSearchBufSize, err := strconv.Atoi(msgStreamSearchBufSizeStr)
|
||||
masterServiceAddr := Params.MasterAddress
|
||||
timeout := 3 * time.Second
|
||||
s.masterServiceClient, err = grcpmasterservice.NewGrpcClient(masterServiceAddr, timeout)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
proxynode.Params.MsgStreamSearchBufSize = int64(msgStreamSearchBufSize)
|
||||
|
||||
msgStreamSearchResultBufSizeStr := config.GetString(MsgStreamSearchResultBufSize)
|
||||
msgStreamSearchResultBufSize, err := strconv.Atoi(msgStreamSearchResultBufSizeStr)
|
||||
err = s.masterServiceClient.Init()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
proxynode.Params.MsgStreamSearchResultBufSize = int64(msgStreamSearchResultBufSize)
|
||||
s.impl.SetMasterClient(s.masterServiceClient)
|
||||
|
||||
msgStreamSearchResultPulsarBufSizeStr := config.GetString(MsgStreamSearchResultPulsarBufSize)
|
||||
msgStreamSearchResultPulsarBufSize, err := strconv.Atoi(msgStreamSearchResultPulsarBufSizeStr)
|
||||
dataServiceAddr := Params.DataServiceAddress
|
||||
s.dataServiceClient = grpcdataservice.NewClient(dataServiceAddr)
|
||||
err = s.dataServiceClient.Init()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
proxynode.Params.MsgStreamSearchResultPulsarBufSize = int64(msgStreamSearchResultPulsarBufSize)
|
||||
|
||||
msgStreamTimeTickBufSizeStr := config.GetString(MsgStreamTimeTickBufSize)
|
||||
msgStreamTimeTickBufSize, err := strconv.Atoi(msgStreamTimeTickBufSizeStr)
|
||||
s.impl.SetDataServiceClient(s.dataServiceClient)
|
||||
|
||||
indexServiceAddr := Params.IndexServerAddress
|
||||
s.indexServiceClient = grpcindexserviceclient.NewClient(indexServiceAddr)
|
||||
err = s.indexServiceClient.Init()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
proxynode.Params.MsgStreamTimeTickBufSize = int64(msgStreamTimeTickBufSize)
|
||||
s.impl.SetIndexServiceClient(s.indexServiceClient)
|
||||
|
||||
maxNameLengthStr := config.GetString(MaxNameLength)
|
||||
maxNameLength, err := strconv.Atoi(maxNameLengthStr)
|
||||
queryServiceAddr := Params.QueryServiceAddress
|
||||
s.queryServiceClient = grpcqueryserviceclient.NewClient(queryServiceAddr)
|
||||
err = s.queryServiceClient.Init()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
proxynode.Params.MaxNameLength = int64(maxNameLength)
|
||||
s.impl.SetQueryServiceClient(s.queryServiceClient)
|
||||
|
||||
maxFieldNumStr := config.GetString(MaxFieldNum)
|
||||
maxFieldNum, err := strconv.Atoi(maxFieldNumStr)
|
||||
if err != nil {
|
||||
proxynode.Params.Init()
|
||||
proxynode.Params.NetworkPort = Params.Port
|
||||
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
|
||||
}
|
||||
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
|
||||
}
|
||||
|
||||
func (s *Server) connectProxyService() 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)
|
||||
}
|
||||
}()
|
||||
|
||||
func (s *Server) start() error {
|
||||
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) {
|
||||
return s.impl.CreateCollection(ctx, request)
|
||||
return s.impl.CreateCollection(request)
|
||||
}
|
||||
|
||||
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) {
|
||||
return s.impl.HasCollection(ctx, request)
|
||||
return s.impl.HasCollection(request)
|
||||
}
|
||||
|
||||
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) {
|
||||
return s.impl.ReleaseCollection(ctx, request)
|
||||
return s.impl.ReleaseCollection(request)
|
||||
}
|
||||
|
||||
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) {
|
||||
return s.impl.GetCollectionStatistics(ctx, request)
|
||||
return s.impl.GetCollectionStatistics(request)
|
||||
}
|
||||
|
||||
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) {
|
||||
return s.impl.CreatePartition(ctx, request)
|
||||
return s.impl.CreatePartition(request)
|
||||
}
|
||||
|
||||
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) {
|
||||
return s.impl.HasPartition(ctx, request)
|
||||
return s.impl.HasPartition(request)
|
||||
}
|
||||
|
||||
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) {
|
||||
return s.impl.ReleasePartitions(ctx, request)
|
||||
return s.impl.ReleasePartitions(request)
|
||||
}
|
||||
|
||||
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) {
|
||||
return s.impl.ShowPartitions(ctx, request)
|
||||
return s.impl.ShowPartitions(request)
|
||||
}
|
||||
|
||||
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) {
|
||||
return s.impl.DescribeIndex(ctx, request)
|
||||
return s.impl.DescribeIndex(request)
|
||||
}
|
||||
|
||||
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) {
|
||||
return s.impl.Insert(ctx, request)
|
||||
return s.impl.Insert(request)
|
||||
}
|
||||
|
||||
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) {
|
||||
return s.impl.Flush(ctx, request)
|
||||
return s.impl.Flush(request)
|
||||
}
|
||||
|
||||
func (s *Server) GetDdChannel(ctx context.Context, request *commonpb.Empty) (*milvuspb.StringResponse, error) {
|
||||
return s.impl.GetDdChannel(ctx, request)
|
||||
return s.impl.GetDdChannel(request)
|
||||
}
|
||||
|
|
|
@ -2,14 +2,14 @@ package grpcproxyservice
|
|||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"time"
|
||||
|
||||
"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/util/retry"
|
||||
)
|
||||
|
||||
type Client struct {
|
||||
|
@ -18,41 +18,40 @@ type Client struct {
|
|||
ctx context.Context
|
||||
}
|
||||
|
||||
func (c *Client) tryConnect() error {
|
||||
if c.proxyServiceClient != nil {
|
||||
func (c *Client) Init() error {
|
||||
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
|
||||
}
|
||||
conn, err := grpc.DialContext(c.ctx, c.address, grpc.WithInsecure(), grpc.WithBlock())
|
||||
err := retry.Retry(10, time.Millisecond*200, connectGrpcFunc)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
c.proxyServiceClient = proxypb.NewProxyServiceClient(conn)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *Client) Start() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *Client) Stop() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
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)
|
||||
}
|
||||
|
||||
func (c *Client) InvalidateCollectionMetaCache(request *proxypb.InvalidateCollMetaCacheRequest) error {
|
||||
var err error
|
||||
err = c.tryConnect()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
_, err = c.proxyServiceClient.InvalidateCollectionMetaCache(c.ctx, request)
|
||||
_, err := c.proxyServiceClient.InvalidateCollectionMetaCache(c.ctx, request)
|
||||
return err
|
||||
}
|
||||
|
||||
func (c *Client) GetTimeTickChannel() (string, error) {
|
||||
err := c.tryConnect()
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
response, err := c.proxyServiceClient.GetTimeTickChannel(c.ctx, &commonpb.Empty{})
|
||||
if err != nil {
|
||||
return "", err
|
||||
|
@ -61,13 +60,13 @@ func (c *Client) GetTimeTickChannel() (string, 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{})
|
||||
}
|
||||
|
||||
func (c *Client) GetStatisticsChannel() (string, error) {
|
||||
return "", nil
|
||||
}
|
||||
|
||||
func NewClient(address string) *Client {
|
||||
return &Client{
|
||||
address: address,
|
||||
|
|
|
@ -9,19 +9,28 @@ import (
|
|||
|
||||
type ParamTable struct {
|
||||
paramtable.BaseTable
|
||||
|
||||
ServiceAddress string
|
||||
ServicePort int
|
||||
}
|
||||
|
||||
var Params ParamTable
|
||||
|
||||
func (pt *ParamTable) Init() {
|
||||
pt.BaseTable.Init()
|
||||
pt.initParams()
|
||||
}
|
||||
|
||||
func (pt *ParamTable) NetworkPort() int {
|
||||
return pt.ParseInt("proxyService.port")
|
||||
func (pt *ParamTable) initParams() {
|
||||
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")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
|
@ -42,5 +51,5 @@ func (pt *ParamTable) NetworkAddress() string {
|
|||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return addr + ":" + port
|
||||
pt.ServiceAddress = addr + ":" + port
|
||||
}
|
||||
|
|
|
@ -3,85 +3,106 @@ package grpcproxyservice
|
|||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"log"
|
||||
"net"
|
||||
"strconv"
|
||||
"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/internalpb2"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/proxypb"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/proxyservice"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/funcutil"
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
type Server struct {
|
||||
ctx context.Context
|
||||
wg sync.WaitGroup
|
||||
impl proxyservice.ProxyService
|
||||
grpcServer *grpc.Server
|
||||
ctx context.Context
|
||||
wg sync.WaitGroup
|
||||
|
||||
grpcServer *grpc.Server
|
||||
grpcErrChan chan error
|
||||
|
||||
impl *proxyservice.ServiceImpl
|
||||
}
|
||||
|
||||
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
|
||||
func NewServer() (*Server, error) {
|
||||
|
||||
server := &Server{
|
||||
ctx: context.Background(),
|
||||
grpcErrChan: make(chan error),
|
||||
}
|
||||
return &milvuspb.StringResponse{
|
||||
Value: channel,
|
||||
}, nil
|
||||
|
||||
var err error
|
||||
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) {
|
||||
return s.impl.GetComponentStates()
|
||||
}
|
||||
func (s *Server) Run() error {
|
||||
|
||||
func CreateProxyServiceServer() (*Server, error) {
|
||||
return &Server{}, nil
|
||||
}
|
||||
if err := s.init(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
func (s *Server) Init() error {
|
||||
s.ctx = context.Background()
|
||||
Params.Init()
|
||||
proxyservice.Params.Init()
|
||||
s.impl, _ = proxyservice.CreateProxyService(s.ctx)
|
||||
s.impl.Init()
|
||||
if err := s.start(); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *Server) Start() error {
|
||||
fmt.Println("proxy service start ...")
|
||||
func (s *Server) init() error {
|
||||
Params.Init()
|
||||
proxyservice.Params.Init()
|
||||
|
||||
s.wg.Add(1)
|
||||
go func() {
|
||||
defer s.wg.Done()
|
||||
s.startGrpcLoop(Params.ServicePort)
|
||||
// wait for grpc server loop start
|
||||
if err := <-s.grpcErrChan; err != nil {
|
||||
return err
|
||||
}
|
||||
s.impl.UpdateStateCode(internalpb2.StateCode_INITIALIZING)
|
||||
|
||||
// TODO: use config
|
||||
fmt.Println("network port: ", Params.NetworkPort())
|
||||
lis, err := net.Listen("tcp", ":"+strconv.Itoa(Params.NetworkPort()))
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
if err := s.impl.Init(); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
s.grpcServer = grpc.NewServer()
|
||||
proxypb.RegisterProxyServiceServer(s.grpcServer, s)
|
||||
milvuspb.RegisterProxyServiceServer(s.grpcServer, s)
|
||||
if err = s.grpcServer.Serve(lis); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}()
|
||||
func (s *Server) startGrpcLoop(grpcPort int) {
|
||||
|
||||
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
|
||||
}
|
||||
|
||||
|
@ -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) {
|
||||
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()
|
||||
}
|
||||
|
|
|
@ -8,15 +8,10 @@ import (
|
|||
"testing"
|
||||
"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/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/stretchr/testify/assert"
|
||||
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/masterpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
|
||||
"go.etcd.io/etcd/clientv3"
|
||||
"go.uber.org/zap"
|
||||
"google.golang.org/grpc"
|
||||
|
@ -1110,110 +1106,110 @@ func TestMaster(t *testing.T) {
|
|||
assert.Equal(t, dropCollectionReq.CollectionName, dropCollectionMsg.DropCollectionRequest.CollectionName)
|
||||
})
|
||||
|
||||
t.Run("TestSegmentManager_RPC", func(t *testing.T) {
|
||||
collName := "test_coll"
|
||||
partitionName := "test_part"
|
||||
schema := &schemapb.CollectionSchema{
|
||||
Name: collName,
|
||||
Description: "test coll",
|
||||
AutoID: false,
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{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"}}},
|
||||
},
|
||||
}
|
||||
schemaBytes, err := proto.Marshal(schema)
|
||||
assert.Nil(t, err)
|
||||
_, err = cli.CreateCollection(ctx, &milvuspb.CreateCollectionRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_kCreateCollection,
|
||||
MsgID: 1,
|
||||
Timestamp: Timestamp(time.Now().Unix()),
|
||||
SourceID: 1,
|
||||
},
|
||||
Schema: schemaBytes,
|
||||
})
|
||||
assert.Nil(t, err)
|
||||
_, err = cli.CreatePartition(ctx, &milvuspb.CreatePartitionRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_kCreatePartition,
|
||||
MsgID: 2,
|
||||
Timestamp: Timestamp(time.Now().Unix()),
|
||||
SourceID: 1,
|
||||
},
|
||||
CollectionName: collName,
|
||||
PartitionName: partitionName,
|
||||
})
|
||||
assert.Nil(t, err)
|
||||
|
||||
resp, err := cli.AssignSegmentID(ctx, &datapb.AssignSegIDRequest{
|
||||
NodeID: 1,
|
||||
PeerRole: typeutil.ProxyNodeRole,
|
||||
SegIDRequests: []*datapb.SegIDRequest{
|
||||
{Count: 10000, ChannelName: "0", CollName: collName, PartitionName: partitionName},
|
||||
},
|
||||
})
|
||||
assert.Nil(t, err)
|
||||
assignments := resp.GetSegIDAssignments()
|
||||
assert.EqualValues(t, 1, len(assignments))
|
||||
assert.EqualValues(t, commonpb.ErrorCode_SUCCESS, assignments[0].Status.ErrorCode)
|
||||
assert.EqualValues(t, collName, assignments[0].CollName)
|
||||
assert.EqualValues(t, partitionName, assignments[0].PartitionName)
|
||||
assert.EqualValues(t, "0", assignments[0].ChannelName)
|
||||
assert.EqualValues(t, uint32(10000), assignments[0].Count)
|
||||
|
||||
// test stats
|
||||
segID := assignments[0].SegID
|
||||
pulsarAddress := Params.PulsarAddress
|
||||
msgStream := pulsarms.NewPulsarMsgStream(ctx, 1024)
|
||||
msgStream.SetPulsarClient(pulsarAddress)
|
||||
msgStream.CreatePulsarProducers([]string{Params.QueryNodeStatsChannelName})
|
||||
msgStream.Start()
|
||||
defer msgStream.Close()
|
||||
|
||||
err = msgStream.Produce(&ms.MsgPack{
|
||||
BeginTs: 102,
|
||||
EndTs: 104,
|
||||
Msgs: []ms.TsMsg{
|
||||
&ms.QueryNodeStatsMsg{
|
||||
QueryNodeStats: internalpb2.QueryNodeStats{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_kQueryNodeStats,
|
||||
SourceID: 1,
|
||||
},
|
||||
SegStats: []*internalpb2.SegmentStats{
|
||||
{SegmentID: segID, MemorySize: 600000000, NumRows: 1000000, RecentlyModified: true},
|
||||
},
|
||||
},
|
||||
BaseMsg: ms.BaseMsg{
|
||||
HashValues: []uint32{0},
|
||||
},
|
||||
},
|
||||
},
|
||||
})
|
||||
assert.Nil(t, err)
|
||||
|
||||
time.Sleep(500 * time.Millisecond)
|
||||
segMeta, err := svr.metaTable.GetSegmentByID(segID)
|
||||
assert.Nil(t, err)
|
||||
assert.EqualValues(t, 1000000, segMeta.GetNumRows())
|
||||
assert.EqualValues(t, int64(600000000), segMeta.GetMemSize())
|
||||
|
||||
reqDrop := milvuspb.DropCollectionRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_kDropCollection,
|
||||
MsgID: 1,
|
||||
Timestamp: Timestamp(time.Now().Unix()),
|
||||
SourceID: 1,
|
||||
},
|
||||
CollectionName: collName,
|
||||
}
|
||||
|
||||
// DropCollection
|
||||
st, err := cli.DropCollection(ctx, &reqDrop)
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, st.ErrorCode, commonpb.ErrorCode_SUCCESS)
|
||||
})
|
||||
//t.Run("TestSegmentManager_RPC", func(t *testing.T) {
|
||||
// collName := "test_coll"
|
||||
// partitionName := "test_part"
|
||||
// schema := &schemapb.CollectionSchema{
|
||||
// Name: collName,
|
||||
// Description: "test coll",
|
||||
// AutoID: false,
|
||||
// Fields: []*schemapb.FieldSchema{
|
||||
// {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"}}},
|
||||
// },
|
||||
// }
|
||||
// schemaBytes, err := proto.Marshal(schema)
|
||||
// assert.Nil(t, err)
|
||||
// _, err = cli.CreateCollection(ctx, &milvuspb.CreateCollectionRequest{
|
||||
// Base: &commonpb.MsgBase{
|
||||
// MsgType: commonpb.MsgType_kCreateCollection,
|
||||
// MsgID: 1,
|
||||
// Timestamp: Timestamp(time.Now().Unix()),
|
||||
// SourceID: 1,
|
||||
// },
|
||||
// Schema: schemaBytes,
|
||||
// })
|
||||
// assert.Nil(t, err)
|
||||
// _, err = cli.CreatePartition(ctx, &milvuspb.CreatePartitionRequest{
|
||||
// Base: &commonpb.MsgBase{
|
||||
// MsgType: commonpb.MsgType_kCreatePartition,
|
||||
// MsgID: 2,
|
||||
// Timestamp: Timestamp(time.Now().Unix()),
|
||||
// SourceID: 1,
|
||||
// },
|
||||
// CollectionName: collName,
|
||||
// PartitionName: partitionName,
|
||||
// })
|
||||
// assert.Nil(t, err)
|
||||
//
|
||||
// resp, err := cli.AssignSegmentID(ctx, &datapb.AssignSegIDRequest{
|
||||
// NodeID: 1,
|
||||
// PeerRole: typeutil.ProxyNodeRole,
|
||||
// SegIDRequests: []*datapb.SegIDRequest{
|
||||
// {Count: 10000, ChannelName: "0", CollName: collName, PartitionName: partitionName},
|
||||
// },
|
||||
// })
|
||||
// assert.Nil(t, err)
|
||||
// assignments := resp.GetSegIDAssignments()
|
||||
// assert.EqualValues(t, 1, len(assignments))
|
||||
// assert.EqualValues(t, commonpb.ErrorCode_SUCCESS, assignments[0].Status.ErrorCode)
|
||||
// assert.EqualValues(t, collName, assignments[0].CollName)
|
||||
// assert.EqualValues(t, partitionName, assignments[0].PartitionName)
|
||||
// assert.EqualValues(t, "0", assignments[0].ChannelName)
|
||||
// assert.EqualValues(t, uint32(10000), assignments[0].Count)
|
||||
//
|
||||
// // test stats
|
||||
// segID := assignments[0].SegID
|
||||
// pulsarAddress := Params.PulsarAddress
|
||||
// msgStream := pulsarms.NewPulsarMsgStream(ctx, 1024)
|
||||
// msgStream.SetPulsarClient(pulsarAddress)
|
||||
// msgStream.CreatePulsarProducers([]string{Params.QueryNodeStatsChannelName})
|
||||
// msgStream.Start()
|
||||
// defer msgStream.Close()
|
||||
//
|
||||
// err = msgStream.Produce(&ms.MsgPack{
|
||||
// BeginTs: 102,
|
||||
// EndTs: 104,
|
||||
// Msgs: []ms.TsMsg{
|
||||
// &ms.QueryNodeStatsMsg{
|
||||
// QueryNodeStats: internalpb2.QueryNodeStats{
|
||||
// Base: &commonpb.MsgBase{
|
||||
// MsgType: commonpb.MsgType_kQueryNodeStats,
|
||||
// SourceID: 1,
|
||||
// },
|
||||
// SegStats: []*internalpb2.SegmentStats{
|
||||
// {SegmentID: segID, MemorySize: 600000000, NumRows: 1000000, RecentlyModified: true},
|
||||
// },
|
||||
// },
|
||||
// BaseMsg: ms.BaseMsg{
|
||||
// HashValues: []uint32{0},
|
||||
// },
|
||||
// },
|
||||
// },
|
||||
// })
|
||||
// assert.Nil(t, err)
|
||||
//
|
||||
// time.Sleep(500 * time.Millisecond)
|
||||
// segMeta, err := svr.metaTable.GetSegmentByID(segID)
|
||||
// assert.Nil(t, err)
|
||||
// assert.EqualValues(t, 1000000, segMeta.GetNumRows())
|
||||
// assert.EqualValues(t, int64(600000000), segMeta.GetMemSize())
|
||||
//
|
||||
// reqDrop := milvuspb.DropCollectionRequest{
|
||||
// Base: &commonpb.MsgBase{
|
||||
// MsgType: commonpb.MsgType_kDropCollection,
|
||||
// MsgID: 1,
|
||||
// Timestamp: Timestamp(time.Now().Unix()),
|
||||
// SourceID: 1,
|
||||
// },
|
||||
// CollectionName: collName,
|
||||
// }
|
||||
//
|
||||
// // DropCollection
|
||||
// st, err := cli.DropCollection(ctx, &reqDrop)
|
||||
// assert.Nil(t, err)
|
||||
// assert.Equal(t, st.ErrorCode, commonpb.ErrorCode_SUCCESS)
|
||||
//})
|
||||
|
||||
cancel()
|
||||
conn.Close()
|
||||
|
|
|
@ -6,7 +6,6 @@ option go_package="github.com/zilliztech/milvus-distributed/internal/proto/maste
|
|||
import "common.proto";
|
||||
import "milvus.proto";
|
||||
import "internal.proto";
|
||||
import "data_service.proto";
|
||||
|
||||
message IDRequest {
|
||||
common.MsgBase base = 1;
|
||||
|
@ -109,14 +108,10 @@ service MasterService {
|
|||
|
||||
rpc CreateIndex(milvus.CreateIndexRequest) returns (common.Status) {}
|
||||
rpc DescribeIndex(milvus.DescribeIndexRequest) returns (milvus.DescribeIndexResponse) {}
|
||||
rpc GetIndexState(milvus.IndexStateRequest) returns (milvus.IndexStateResponse) {}
|
||||
|
||||
rpc AllocTimestamp(TsoRequest) returns (TsoResponse) {}
|
||||
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 GetTimeTickChannelRPC(common.Empty) returns (milvus.StringResponse) {}
|
||||
|
|
|
@ -8,7 +8,6 @@ import (
|
|||
fmt "fmt"
|
||||
proto "github.com/golang/protobuf/proto"
|
||||
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"
|
||||
milvuspb "github.com/zilliztech/milvus-distributed/internal/proto/milvuspb"
|
||||
grpc "google.golang.org/grpc"
|
||||
|
@ -242,52 +241,49 @@ func init() {
|
|||
func init() { proto.RegisterFile("master.proto", fileDescriptor_f9c348dec43a6705) }
|
||||
|
||||
var fileDescriptor_f9c348dec43a6705 = []byte{
|
||||
// 715 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x96, 0xeb, 0x6e, 0xd3, 0x30,
|
||||
0x18, 0x86, 0x77, 0x62, 0x68, 0xdf, 0x7a, 0x98, 0x0c, 0x88, 0xa9, 0x4c, 0x6c, 0x14, 0xb1, 0x75,
|
||||
0x07, 0x52, 0xb4, 0xdd, 0x00, 0x6b, 0x83, 0xba, 0x4a, 0x4c, 0x9a, 0xd2, 0xf2, 0x83, 0xc1, 0x34,
|
||||
0x25, 0xa9, 0xd5, 0x5a, 0x24, 0x76, 0x88, 0xdd, 0x0d, 0x76, 0x0b, 0xdc, 0x34, 0x8a, 0x93, 0x38,
|
||||
0x49, 0x97, 0xac, 0x41, 0xec, 0xa7, 0xed, 0xd7, 0xcf, 0xfb, 0x9d, 0x14, 0x07, 0x2a, 0xae, 0xc9,
|
||||
0x05, 0xf6, 0x35, 0xcf, 0x67, 0x82, 0xa1, 0x67, 0x2e, 0x71, 0x6e, 0xa6, 0x3c, 0x5c, 0x69, 0xe1,
|
||||
0x51, 0xa3, 0x62, 0x33, 0xd7, 0x65, 0x34, 0xdc, 0x6c, 0x54, 0xd2, 0x92, 0x46, 0x8d, 0x50, 0x81,
|
||||
0x7d, 0x6a, 0x3a, 0xd1, 0x1a, 0x8d, 0x4c, 0x61, 0x5e, 0x73, 0xec, 0xdf, 0x10, 0x1b, 0x87, 0x7b,
|
||||
0xcd, 0x01, 0xac, 0xf5, 0x75, 0x03, 0xff, 0x9c, 0x62, 0x2e, 0xd0, 0x07, 0x58, 0xb1, 0x4c, 0x8e,
|
||||
0x37, 0x17, 0x77, 0x16, 0x5b, 0xeb, 0xc7, 0x5b, 0x5a, 0xc6, 0x30, 0x32, 0x3a, 0xe7, 0xe3, 0x8e,
|
||||
0xc9, 0xb1, 0x21, 0x95, 0xe8, 0x39, 0x3c, 0xb1, 0xd9, 0x94, 0x8a, 0xcd, 0xa5, 0x9d, 0xc5, 0x56,
|
||||
0xd5, 0x08, 0x17, 0xcd, 0x31, 0x40, 0x00, 0xe5, 0x1e, 0xa3, 0x1c, 0xa3, 0x13, 0x58, 0xe5, 0xc2,
|
||||
0x14, 0x53, 0x1e, 0x71, 0x5f, 0xe5, 0x72, 0x07, 0x52, 0x62, 0x44, 0x52, 0x54, 0x83, 0xa5, 0xbe,
|
||||
0x2e, 0xa9, 0xcb, 0xc6, 0x52, 0x5f, 0x4f, 0x8c, 0x96, 0xd3, 0x46, 0x43, 0x80, 0x21, 0x67, 0x8f,
|
||||
0x10, 0x7e, 0x86, 0x7a, 0x03, 0xeb, 0x92, 0xfa, 0x3f, 0xf1, 0x6f, 0xc1, 0x9a, 0x20, 0x2e, 0xe6,
|
||||
0xc2, 0x74, 0x3d, 0x99, 0xc6, 0x8a, 0x91, 0x6c, 0xe4, 0xfb, 0x1e, 0xff, 0xd9, 0x80, 0xea, 0xb9,
|
||||
0x6c, 0xeb, 0x20, 0xec, 0x11, 0xba, 0x86, 0x8d, 0xae, 0x8f, 0x4d, 0x81, 0xbb, 0xcc, 0x71, 0xb0,
|
||||
0x2d, 0x08, 0xa3, 0xe8, 0x28, 0x6b, 0x1f, 0x2d, 0x66, 0x65, 0x51, 0x4d, 0x1a, 0x0f, 0x05, 0xdb,
|
||||
0x5c, 0x40, 0xdf, 0xa0, 0xa6, 0xfb, 0xcc, 0x4b, 0xe1, 0x0f, 0x72, 0xf1, 0x59, 0x51, 0x49, 0xf8,
|
||||
0x35, 0x54, 0xcf, 0x4c, 0x9e, 0x62, 0xef, 0xe7, 0xb2, 0x33, 0x9a, 0x18, 0xfd, 0x26, 0x57, 0xda,
|
||||
0x61, 0xcc, 0x89, 0xfb, 0xd2, 0x5c, 0x40, 0xb7, 0x80, 0x74, 0xcc, 0x6d, 0x9f, 0x58, 0xe9, 0x02,
|
||||
0x69, 0xf9, 0x19, 0xdc, 0x13, 0xc6, 0x56, 0xed, 0xd2, 0x7a, 0x65, 0xec, 0x40, 0x7d, 0x30, 0x61,
|
||||
0xb7, 0xc9, 0x19, 0x2f, 0xa8, 0x5b, 0x56, 0x15, 0x3b, 0x1e, 0x96, 0xd2, 0x2a, 0xb7, 0x2b, 0xa8,
|
||||
0x87, 0xed, 0xbd, 0x30, 0x7d, 0x41, 0x64, 0x8e, 0x87, 0x0f, 0x0c, 0x81, 0x52, 0x95, 0x6c, 0xd3,
|
||||
0x57, 0xa8, 0x06, 0xed, 0x4d, 0xe0, 0xfb, 0x85, 0x23, 0xf0, 0xaf, 0xe8, 0x2b, 0xa8, 0x9c, 0x99,
|
||||
0x3c, 0x21, 0xb7, 0x8a, 0x06, 0xe0, 0x1e, 0xb8, 0x54, 0xff, 0x09, 0xd4, 0x82, 0xa2, 0xa9, 0xcb,
|
||||
0xbc, 0x20, 0xf4, 0x8c, 0x28, 0x76, 0x38, 0x28, 0x23, 0x55, 0x56, 0x14, 0xea, 0xf1, 0x44, 0x0c,
|
||||
0xf0, 0xd8, 0xc5, 0x54, 0x14, 0xf4, 0x60, 0x46, 0x15, 0xbb, 0x1d, 0x95, 0x13, 0x2b, 0x3f, 0x1b,
|
||||
0x2a, 0x41, 0x28, 0xd1, 0x01, 0x47, 0x7b, 0x85, 0xd1, 0xce, 0x18, 0xb5, 0xe6, 0x0b, 0x95, 0xc9,
|
||||
0x17, 0x58, 0x0f, 0x47, 0xa6, 0x4f, 0x47, 0xf8, 0x57, 0x81, 0x47, 0x4a, 0x51, 0xb2, 0xeb, 0x13,
|
||||
0xa8, 0xc6, 0x89, 0x85, 0xe0, 0xfd, 0x07, 0x93, 0xcf, 0xa0, 0x0f, 0xca, 0x48, 0x55, 0x02, 0x16,
|
||||
0x54, 0x7b, 0x58, 0xc8, 0xdd, 0xc0, 0x1d, 0xa3, 0xdd, 0xdc, 0xeb, 0x89, 0x20, 0xb6, 0xd9, 0x9b,
|
||||
0xab, 0x4b, 0x15, 0xa9, 0x76, 0xea, 0x38, 0xcc, 0x1e, 0xaa, 0xaf, 0xf7, 0xb6, 0x96, 0xf3, 0x12,
|
||||
0x6b, 0xc9, 0x43, 0xd4, 0xd8, 0x29, 0x16, 0x28, 0xec, 0x67, 0x78, 0x2a, 0xb1, 0x7d, 0x1d, 0xbd,
|
||||
0xce, 0x95, 0xab, 0x67, 0xb9, 0xb1, 0x5d, 0x78, 0x9e, 0x2a, 0x44, 0xfd, 0x94, 0x73, 0x32, 0xa6,
|
||||
0x51, 0x93, 0xfb, 0x3a, 0x7a, 0x97, 0xbd, 0x15, 0xbc, 0xfd, 0x9a, 0xd2, 0x24, 0xf0, 0xdd, 0x79,
|
||||
0xb2, 0xd4, 0x67, 0xe8, 0x45, 0x0f, 0x8b, 0x2e, 0x73, 0x3d, 0x46, 0x31, 0x15, 0xb2, 0x4e, 0xdc,
|
||||
0xb8, 0xe8, 0xa2, 0x46, 0xee, 0x38, 0x7c, 0x72, 0x3d, 0xf1, 0x7b, 0x16, 0xaf, 0xfe, 0x48, 0x66,
|
||||
0x30, 0xcd, 0x05, 0x74, 0x29, 0xf1, 0x41, 0x95, 0x87, 0xc4, 0xfe, 0xd1, 0x9d, 0x98, 0x94, 0x62,
|
||||
0x67, 0x1e, 0xfe, 0x6d, 0xfe, 0xb4, 0x0b, 0x9f, 0xd0, 0x71, 0x2a, 0xf4, 0x21, 0xd4, 0x7b, 0x58,
|
||||
0xe8, 0xa3, 0xc7, 0xa5, 0x7e, 0x87, 0x97, 0x3d, 0x2c, 0x13, 0x20, 0x5c, 0x10, 0x9b, 0x3f, 0x2a,
|
||||
0xbd, 0xd3, 0xb9, 0xfc, 0x38, 0x26, 0x62, 0x32, 0xb5, 0x82, 0xdb, 0xed, 0x3b, 0xe2, 0x38, 0xe4,
|
||||
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,
|
||||
// 662 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x95, 0xe9, 0x6e, 0xd3, 0x4c,
|
||||
0x14, 0x86, 0xbb, 0x7d, 0xfd, 0xd4, 0xd3, 0x2c, 0x68, 0x00, 0x51, 0x99, 0x8a, 0x96, 0x20, 0x41,
|
||||
0xba, 0x90, 0xa0, 0xf6, 0x06, 0x20, 0x31, 0x6a, 0x23, 0x51, 0xa9, 0x72, 0xc2, 0x0f, 0x0a, 0x55,
|
||||
0xe5, 0x38, 0x47, 0xc9, 0x08, 0x7b, 0xc6, 0x78, 0xc6, 0x2d, 0xf4, 0xf6, 0xb8, 0x31, 0xe4, 0xf1,
|
||||
0x9e, 0xda, 0xad, 0x11, 0xf9, 0x39, 0x3e, 0xef, 0x3c, 0xef, 0xd9, 0x92, 0x81, 0x9a, 0x63, 0x0a,
|
||||
0x89, 0x5e, 0xc7, 0xf5, 0xb8, 0xe4, 0xe4, 0xb1, 0x43, 0xed, 0x6b, 0x5f, 0x84, 0xa7, 0x4e, 0x18,
|
||||
0xd2, 0x6a, 0x16, 0x77, 0x1c, 0xce, 0xc2, 0x8f, 0x5a, 0x2d, 0x2b, 0xd1, 0x1a, 0x94, 0x49, 0xf4,
|
||||
0x98, 0x69, 0x87, 0xe7, 0xd6, 0x10, 0x36, 0x06, 0xba, 0x81, 0x3f, 0x7c, 0x14, 0x92, 0xbc, 0x83,
|
||||
0xb5, 0xb1, 0x29, 0x70, 0x6b, 0x79, 0x77, 0xb9, 0xbd, 0x79, 0xb4, 0xdd, 0xc9, 0xc1, 0x23, 0xe8,
|
||||
0x99, 0x98, 0xf6, 0x4c, 0x81, 0x86, 0x52, 0x92, 0x27, 0xf0, 0x9f, 0xc5, 0x7d, 0x26, 0xb7, 0x56,
|
||||
0x76, 0x97, 0xdb, 0x75, 0x23, 0x3c, 0xb4, 0xa6, 0x00, 0x01, 0x54, 0xb8, 0x9c, 0x09, 0x24, 0xc7,
|
||||
0xb0, 0x2e, 0xa4, 0x29, 0x7d, 0x11, 0x71, 0x9f, 0x17, 0x72, 0x87, 0x4a, 0x62, 0x44, 0x52, 0xd2,
|
||||
0x80, 0x95, 0x81, 0xae, 0xa8, 0xab, 0xc6, 0xca, 0x40, 0x4f, 0x8d, 0x56, 0xb3, 0x46, 0x23, 0x80,
|
||||
0x91, 0xe0, 0x0b, 0x48, 0x3f, 0x47, 0xbd, 0x86, 0x4d, 0x45, 0xfd, 0x97, 0xfc, 0xb7, 0x61, 0x43,
|
||||
0x52, 0x07, 0x85, 0x34, 0x1d, 0x57, 0x95, 0xb1, 0x66, 0xa4, 0x1f, 0x8a, 0x7d, 0x8f, 0x7e, 0x37,
|
||||
0xa0, 0x7e, 0xa6, 0x46, 0x38, 0x44, 0xef, 0x9a, 0x5a, 0x48, 0xae, 0xe0, 0x51, 0xdf, 0x43, 0x53,
|
||||
0x62, 0x9f, 0xdb, 0x36, 0x5a, 0x92, 0x72, 0x46, 0x0e, 0xf3, 0xf6, 0xd1, 0x61, 0x5e, 0x16, 0xf5,
|
||||
0x44, 0xbb, 0x2f, 0xd9, 0xd6, 0x12, 0xf9, 0x0a, 0x0d, 0xdd, 0xe3, 0x6e, 0x06, 0xbf, 0x5f, 0x88,
|
||||
0xcf, 0x8b, 0x2a, 0xc2, 0xaf, 0xa0, 0x7e, 0x6a, 0x8a, 0x0c, 0x7b, 0xaf, 0x90, 0x9d, 0xd3, 0xc4,
|
||||
0xe8, 0x97, 0x85, 0xd2, 0x1e, 0xe7, 0x76, 0x3c, 0x97, 0xd6, 0x12, 0xb9, 0x01, 0xa2, 0xa3, 0xb0,
|
||||
0x3c, 0x3a, 0xce, 0x36, 0xa8, 0x53, 0x5c, 0xc1, 0x1d, 0x61, 0x6c, 0xd5, 0xad, 0xac, 0x4f, 0x8c,
|
||||
0x6d, 0x68, 0x0e, 0x67, 0xfc, 0x26, 0x8d, 0x89, 0x92, 0xbe, 0xe5, 0x55, 0xb1, 0xe3, 0x41, 0x25,
|
||||
0x6d, 0xe2, 0x76, 0x09, 0xcd, 0x70, 0xbc, 0xe7, 0xa6, 0x27, 0xa9, 0xaa, 0xf1, 0xe0, 0x9e, 0x25,
|
||||
0x48, 0x54, 0x15, 0xc7, 0xf4, 0x05, 0xea, 0xc1, 0x78, 0x53, 0xf8, 0x5e, 0xe9, 0x0a, 0xfc, 0x2d,
|
||||
0xfa, 0x12, 0x6a, 0xa7, 0xa6, 0x48, 0xc9, 0xed, 0xb2, 0x05, 0xb8, 0x03, 0xae, 0x34, 0x7f, 0x0a,
|
||||
0x8d, 0xa0, 0x69, 0xc9, 0x65, 0x51, 0x92, 0x7a, 0x4e, 0x14, 0x3b, 0xec, 0x57, 0x91, 0x26, 0x56,
|
||||
0x0c, 0x9a, 0xf1, 0x46, 0x0c, 0x71, 0xea, 0x20, 0x93, 0x25, 0x33, 0x98, 0x53, 0xc5, 0x6e, 0x87,
|
||||
0xd5, 0xc4, 0x89, 0x9f, 0x05, 0xb5, 0x20, 0x95, 0x28, 0x20, 0xc8, 0x9b, 0xd2, 0x6c, 0xe7, 0x8c,
|
||||
0xda, 0x0f, 0x0b, 0x13, 0x93, 0xcf, 0xb0, 0x19, 0xae, 0xcc, 0x80, 0x4d, 0xf0, 0x67, 0x89, 0x47,
|
||||
0x46, 0x51, 0x71, 0xea, 0x33, 0xa8, 0xc7, 0x85, 0x85, 0xe0, 0xbd, 0x7b, 0x8b, 0xcf, 0xa1, 0xf7,
|
||||
0xab, 0x48, 0x33, 0x05, 0x34, 0x3e, 0xd8, 0x36, 0xb7, 0x46, 0xc9, 0x3f, 0xeb, 0x4e, 0xa7, 0xe0,
|
||||
0x45, 0xec, 0xa4, 0x8f, 0x84, 0xb6, 0x5b, 0x2e, 0x48, 0xb0, 0x9f, 0xe0, 0x7f, 0x85, 0x1d, 0xe8,
|
||||
0xe4, 0x45, 0xa1, 0x3c, 0x79, 0x32, 0xb5, 0x9d, 0xd2, 0x78, 0xe6, 0xe7, 0xfb, 0xf4, 0x04, 0x65,
|
||||
0x9f, 0x3b, 0x2e, 0x67, 0xc8, 0x64, 0xd0, 0x26, 0x14, 0xc6, 0x79, 0x9f, 0x68, 0x85, 0x6d, 0xfc,
|
||||
0xe8, 0xb8, 0xf2, 0x97, 0xf6, 0x3a, 0x1f, 0x4b, 0x5e, 0xed, 0x39, 0x4c, 0x6b, 0x89, 0x5c, 0x28,
|
||||
0x7c, 0xd0, 0x81, 0x11, 0xb5, 0xbe, 0xf7, 0x67, 0x26, 0x63, 0x68, 0x3f, 0x84, 0x7f, 0x55, 0xbc,
|
||||
0x25, 0xd2, 0xa3, 0x6c, 0x9a, 0x49, 0x7d, 0x04, 0xcd, 0x13, 0x94, 0xfa, 0x64, 0xb1, 0xd4, 0x6f,
|
||||
0xf0, 0xec, 0x04, 0x55, 0x01, 0x54, 0x48, 0x6a, 0x89, 0x85, 0xd2, 0x7b, 0xbd, 0x8b, 0xf7, 0x53,
|
||||
0x2a, 0x67, 0xfe, 0x38, 0xb8, 0xdd, 0xbd, 0xa5, 0xb6, 0x4d, 0x6f, 0x25, 0x5a, 0xb3, 0x6e, 0x78,
|
||||
0xe1, 0xed, 0x84, 0x0a, 0xe9, 0xd1, 0xb1, 0x2f, 0x71, 0xd2, 0x8d, 0xdb, 0xda, 0x55, 0xc8, 0x6e,
|
||||
0x38, 0x3d, 0x77, 0x3c, 0x5e, 0x57, 0xe7, 0xe3, 0x3f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x41, 0x9e,
|
||||
0x3e, 0x16, 0x6d, 0x09, 0x00, 0x00,
|
||||
}
|
||||
|
||||
// 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)
|
||||
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)
|
||||
GetIndexState(ctx context.Context, in *milvuspb.IndexStateRequest, opts ...grpc.CallOption) (*milvuspb.IndexStateResponse, error)
|
||||
AllocTimestamp(ctx context.Context, in *TsoRequest, opts ...grpc.CallOption) (*TsoResponse, 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)
|
||||
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)
|
||||
|
@ -497,15 +490,6 @@ func (c *masterServiceClient) DescribeIndex(ctx context.Context, in *milvuspb.De
|
|||
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) {
|
||||
out := new(TsoResponse)
|
||||
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
|
||||
}
|
||||
|
||||
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) {
|
||||
out := new(internalpb2.ComponentStates)
|
||||
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)
|
||||
CreateIndex(context.Context, *milvuspb.CreateIndexRequest) (*commonpb.Status, error)
|
||||
DescribeIndex(context.Context, *milvuspb.DescribeIndexRequest) (*milvuspb.DescribeIndexResponse, error)
|
||||
GetIndexState(context.Context, *milvuspb.IndexStateRequest) (*milvuspb.IndexStateResponse, error)
|
||||
AllocTimestamp(context.Context, *TsoRequest) (*TsoResponse, 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)
|
||||
GetTimeTickChannelRPC(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) {
|
||||
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) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method AllocTimestamp not implemented")
|
||||
}
|
||||
func (*UnimplementedMasterServiceServer) AllocID(ctx context.Context, req *IDRequest) (*IDResponse, error) {
|
||||
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) {
|
||||
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)
|
||||
}
|
||||
|
||||
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) {
|
||||
in := new(TsoRequest)
|
||||
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)
|
||||
}
|
||||
|
||||
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) {
|
||||
in := new(commonpb.Empty)
|
||||
if err := dec(in); err != nil {
|
||||
|
@ -1147,10 +1077,6 @@ var _MasterService_serviceDesc = grpc.ServiceDesc{
|
|||
MethodName: "DescribeIndex",
|
||||
Handler: _MasterService_DescribeIndex_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "GetIndexState",
|
||||
Handler: _MasterService_GetIndexState_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "AllocTimestamp",
|
||||
Handler: _MasterService_AllocTimestamp_Handler,
|
||||
|
@ -1159,10 +1085,6 @@ var _MasterService_serviceDesc = grpc.ServiceDesc{
|
|||
MethodName: "AllocID",
|
||||
Handler: _MasterService_AllocID_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "AssignSegmentID",
|
||||
Handler: _MasterService_AssignSegmentID_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "GetComponentStatesRPC",
|
||||
Handler: _MasterService_GetComponentStatesRPC_Handler,
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -20,12 +20,17 @@ const (
|
|||
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) {
|
||||
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)
|
||||
ctx := context.Background()
|
||||
cct := &CreateCollectionTask{
|
||||
Condition: NewTaskCondition(ctx),
|
||||
CreateCollectionRequest: request,
|
||||
|
@ -62,8 +67,9 @@ func (node *NodeImpl) CreateCollection(ctx context.Context, request *milvuspb.Cr
|
|||
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)
|
||||
ctx := context.Background()
|
||||
dct := &DropCollectionTask{
|
||||
Condition: NewTaskCondition(ctx),
|
||||
DropCollectionRequest: request,
|
||||
|
@ -100,8 +106,9 @@ func (node *NodeImpl) DropCollection(ctx context.Context, request *milvuspb.Drop
|
|||
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)
|
||||
ctx := context.Background()
|
||||
hct := &HasCollectionTask{
|
||||
Condition: NewTaskCondition(ctx),
|
||||
HasCollectionRequest: request,
|
||||
|
@ -142,16 +149,17 @@ func (node *NodeImpl) HasCollection(ctx context.Context, request *milvuspb.HasCo
|
|||
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")
|
||||
}
|
||||
|
||||
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")
|
||||
}
|
||||
|
||||
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)
|
||||
ctx := context.Background()
|
||||
dct := &DescribeCollectionTask{
|
||||
Condition: NewTaskCondition(ctx),
|
||||
DescribeCollectionRequest: request,
|
||||
|
@ -192,12 +200,13 @@ func (node *NodeImpl) DescribeCollection(ctx context.Context, request *milvuspb.
|
|||
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")
|
||||
}
|
||||
|
||||
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")
|
||||
ctx := context.Background()
|
||||
sct := &ShowCollectionsTask{
|
||||
Condition: NewTaskCondition(ctx),
|
||||
ShowCollectionRequest: request,
|
||||
|
@ -238,8 +247,9 @@ func (node *NodeImpl) ShowCollections(ctx context.Context, request *milvuspb.Sho
|
|||
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)
|
||||
ctx := context.Background()
|
||||
cpt := &CreatePartitionTask{
|
||||
Condition: NewTaskCondition(ctx),
|
||||
CreatePartitionRequest: request,
|
||||
|
@ -276,8 +286,9 @@ func (node *NodeImpl) CreatePartition(ctx context.Context, request *milvuspb.Cre
|
|||
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)
|
||||
ctx := context.Background()
|
||||
dpt := &DropPartitionTask{
|
||||
Condition: NewTaskCondition(ctx),
|
||||
DropPartitionRequest: request,
|
||||
|
@ -315,8 +326,9 @@ func (node *NodeImpl) DropPartition(ctx context.Context, request *milvuspb.DropP
|
|||
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)
|
||||
ctx := context.Background()
|
||||
hpt := &HasPartitionTask{
|
||||
Condition: NewTaskCondition(ctx),
|
||||
HasPartitionRequest: request,
|
||||
|
@ -360,20 +372,21 @@ func (node *NodeImpl) HasPartition(ctx context.Context, request *milvuspb.HasPar
|
|||
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")
|
||||
}
|
||||
|
||||
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")
|
||||
}
|
||||
|
||||
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")
|
||||
}
|
||||
|
||||
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)
|
||||
ctx := context.Background()
|
||||
spt := &ShowPartitionsTask{
|
||||
Condition: NewTaskCondition(ctx),
|
||||
ShowPartitionRequest: request,
|
||||
|
@ -416,8 +429,9 @@ func (node *NodeImpl) ShowPartitions(ctx context.Context, request *milvuspb.Show
|
|||
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)
|
||||
ctx := context.Background()
|
||||
cit := &CreateIndexTask{
|
||||
Condition: NewTaskCondition(ctx),
|
||||
CreateIndexRequest: request,
|
||||
|
@ -455,8 +469,9 @@ func (node *NodeImpl) CreateIndex(ctx context.Context, request *milvuspb.CreateI
|
|||
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)
|
||||
ctx := context.Background()
|
||||
dit := &DescribeIndexTask{
|
||||
Condition: NewTaskCondition(ctx),
|
||||
DescribeIndexRequest: request,
|
||||
|
@ -498,8 +513,9 @@ func (node *NodeImpl) DescribeIndex(ctx context.Context, request *milvuspb.Descr
|
|||
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)
|
||||
ctx := context.Background()
|
||||
dipt := &GetIndexStateTask{
|
||||
Condition: NewTaskCondition(ctx),
|
||||
IndexStateRequest: request,
|
||||
|
@ -540,7 +556,8 @@ func (node *NodeImpl) GetIndexState(ctx context.Context, request *milvuspb.Index
|
|||
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")
|
||||
defer span.Finish()
|
||||
span.SetTag("collection name", request.CollectionName)
|
||||
|
@ -607,7 +624,8 @@ func (node *NodeImpl) Insert(ctx context.Context, request *milvuspb.InsertReques
|
|||
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")
|
||||
defer span.Finish()
|
||||
span.SetTag("collection name", request.CollectionName)
|
||||
|
@ -664,10 +682,10 @@ func (node *NodeImpl) Search(ctx context.Context, request *milvuspb.SearchReques
|
|||
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")
|
||||
}
|
||||
|
||||
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")
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -1,46 +1,93 @@
|
|||
package proxynode
|
||||
|
||||
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/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/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 {
|
||||
Init() error
|
||||
Start() 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)
|
||||
DropCollection(ctx context.Context, request *milvuspb.DropCollectionRequest) (*commonpb.Status, error)
|
||||
HasCollection(ctx context.Context, request *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error)
|
||||
LoadCollection(ctx context.Context, request *milvuspb.LoadCollectionRequest) (*commonpb.Status, error)
|
||||
ReleaseCollection(ctx context.Context, request *milvuspb.ReleaseCollectionRequest) (*commonpb.Status, error)
|
||||
DescribeCollection(ctx context.Context, request *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error)
|
||||
GetCollectionStatistics(ctx context.Context, request *milvuspb.CollectionStatsRequest) (*milvuspb.CollectionStatsResponse, error)
|
||||
ShowCollections(ctx context.Context, request *milvuspb.ShowCollectionRequest) (*milvuspb.ShowCollectionResponse, error)
|
||||
CreateCollection(request *milvuspb.CreateCollectionRequest) (*commonpb.Status, error)
|
||||
DropCollection(request *milvuspb.DropCollectionRequest) (*commonpb.Status, error)
|
||||
HasCollection(request *milvuspb.HasCollectionRequest) (*milvuspb.BoolResponse, error)
|
||||
LoadCollection(request *milvuspb.LoadCollectionRequest) (*commonpb.Status, error)
|
||||
ReleaseCollection(request *milvuspb.ReleaseCollectionRequest) (*commonpb.Status, error)
|
||||
DescribeCollection(request *milvuspb.DescribeCollectionRequest) (*milvuspb.DescribeCollectionResponse, error)
|
||||
GetCollectionStatistics(request *milvuspb.CollectionStatsRequest) (*milvuspb.CollectionStatsResponse, error)
|
||||
ShowCollections(request *milvuspb.ShowCollectionRequest) (*milvuspb.ShowCollectionResponse, error)
|
||||
|
||||
CreatePartition(ctx context.Context, request *milvuspb.CreatePartitionRequest) (*commonpb.Status, error)
|
||||
DropPartition(ctx context.Context, request *milvuspb.DropPartitionRequest) (*commonpb.Status, error)
|
||||
HasPartition(ctx context.Context, request *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error)
|
||||
LoadPartitions(ctx context.Context, request *milvuspb.LoadPartitonRequest) (*commonpb.Status, error)
|
||||
ReleasePartitions(ctx context.Context, request *milvuspb.ReleasePartitionRequest) (*commonpb.Status, error)
|
||||
GetPartitionStatistics(ctx context.Context, request *milvuspb.PartitionStatsRequest) (*milvuspb.PartitionStatsResponse, error)
|
||||
ShowPartitions(ctx context.Context, request *milvuspb.ShowPartitionRequest) (*milvuspb.ShowPartitionResponse, error)
|
||||
CreatePartition(request *milvuspb.CreatePartitionRequest) (*commonpb.Status, error)
|
||||
DropPartition(request *milvuspb.DropPartitionRequest) (*commonpb.Status, error)
|
||||
HasPartition(request *milvuspb.HasPartitionRequest) (*milvuspb.BoolResponse, error)
|
||||
LoadPartitions(request *milvuspb.LoadPartitonRequest) (*commonpb.Status, error)
|
||||
ReleasePartitions(request *milvuspb.ReleasePartitionRequest) (*commonpb.Status, error)
|
||||
GetPartitionStatistics(request *milvuspb.PartitionStatsRequest) (*milvuspb.PartitionStatsResponse, error)
|
||||
ShowPartitions(request *milvuspb.ShowPartitionRequest) (*milvuspb.ShowPartitionResponse, error)
|
||||
|
||||
CreateIndex(ctx context.Context, request *milvuspb.CreateIndexRequest) (*commonpb.Status, error)
|
||||
DescribeIndex(ctx context.Context, request *milvuspb.DescribeIndexRequest) (*milvuspb.DescribeIndexResponse, error)
|
||||
GetIndexState(ctx context.Context, request *milvuspb.IndexStateRequest) (*milvuspb.IndexStateResponse, error)
|
||||
CreateIndex(request *milvuspb.CreateIndexRequest) (*commonpb.Status, error)
|
||||
DescribeIndex(request *milvuspb.DescribeIndexRequest) (*milvuspb.DescribeIndexResponse, error)
|
||||
GetIndexState(request *milvuspb.IndexStateRequest) (*milvuspb.IndexStateResponse, error)
|
||||
|
||||
Insert(ctx context.Context, request *milvuspb.InsertRequest) (*milvuspb.InsertResponse, error)
|
||||
Search(ctx context.Context, request *milvuspb.SearchRequest) (*milvuspb.SearchResults, error)
|
||||
Flush(ctx context.Context, request *milvuspb.FlushRequest) (*commonpb.Status, error)
|
||||
Insert(request *milvuspb.InsertRequest) (*milvuspb.InsertResponse, error)
|
||||
Search(request *milvuspb.SearchRequest) (*milvuspb.SearchResults, 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)
|
||||
}
|
||||
|
|
|
@ -1,25 +1,33 @@
|
|||
package proxynode
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"log"
|
||||
"net"
|
||||
"os"
|
||||
"strconv"
|
||||
"strings"
|
||||
"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"
|
||||
)
|
||||
|
||||
const (
|
||||
StartParamsKey = "START_PARAMS"
|
||||
)
|
||||
|
||||
type ParamTable struct {
|
||||
paramtable.BaseTable
|
||||
|
||||
NetworkPort int
|
||||
NetworkAddress string
|
||||
ProxyServiceAddress string
|
||||
MasterAddress string
|
||||
PulsarAddress string
|
||||
IndexServerAddress string
|
||||
NetworkPort int
|
||||
IP string
|
||||
NetworkAddress string
|
||||
|
||||
MasterAddress string
|
||||
PulsarAddress string
|
||||
|
||||
QueryNodeNum int
|
||||
QueryNodeIDList []UniqueID
|
||||
ProxyID UniqueID
|
||||
|
@ -45,15 +53,152 @@ type ParamTable struct {
|
|||
|
||||
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() {
|
||||
pt.BaseTable.Init()
|
||||
pt.initParams()
|
||||
}
|
||||
|
||||
pt.initNetworkPort()
|
||||
pt.initNetworkAddress()
|
||||
pt.initProxyServiceAddress()
|
||||
pt.initMasterAddress()
|
||||
func (pt *ParamTable) initParams() {
|
||||
pt.initPulsarAddress()
|
||||
pt.initIndexServerAddress()
|
||||
pt.initQueryNodeIDList()
|
||||
pt.initQueryNodeNum()
|
||||
pt.initProxyID()
|
||||
|
@ -77,72 +222,6 @@ func (pt *ParamTable) Init() {
|
|||
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() {
|
||||
ret, err := pt.Load("_PulsarAddress")
|
||||
if err != nil {
|
||||
|
@ -151,31 +230,6 @@ func (pt *ParamTable) initPulsarAddress() {
|
|||
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() {
|
||||
pt.QueryNodeNum = len(pt.QueryNodeIDList)
|
||||
}
|
||||
|
|
|
@ -8,6 +8,12 @@ import (
|
|||
"sync"
|
||||
"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/proto/internalpb2"
|
||||
|
@ -32,17 +38,20 @@ type NodeImpl struct {
|
|||
ip string
|
||||
port int
|
||||
|
||||
masterClient MasterClientInterface
|
||||
stateCode internalpb2.StateCode
|
||||
|
||||
masterClient MasterClient
|
||||
indexServiceClient IndexServiceClient
|
||||
queryServiceClient QueryServiceClient
|
||||
dataServiceClient DataServiceClient
|
||||
proxyServiceClient ProxyServiceClient
|
||||
queryServiceClient QueryServiceClient
|
||||
|
||||
sched *TaskScheduler
|
||||
tick *timeTick
|
||||
|
||||
idAllocator *allocator.IDAllocator
|
||||
tsoAllocator *allocator.TimestampAllocator
|
||||
segAssigner *allocator.SegIDAssigner
|
||||
segAssigner *SegIDAssigner
|
||||
|
||||
manipulationMsgStream *pulsarms.PulsarMsgStream
|
||||
queryMsgStream *pulsarms.PulsarMsgStream
|
||||
|
@ -55,7 +64,7 @@ type NodeImpl struct {
|
|||
closeCallbacks []func()
|
||||
}
|
||||
|
||||
func CreateProxyNodeImpl(ctx context.Context) (*NodeImpl, error) {
|
||||
func NewProxyNodeImpl(ctx context.Context) (*NodeImpl, error) {
|
||||
rand.Seed(time.Now().UnixNano())
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
node := &NodeImpl{
|
||||
|
@ -64,43 +73,111 @@ func CreateProxyNodeImpl(ctx context.Context) (*NodeImpl, error) {
|
|||
}
|
||||
|
||||
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 {
|
||||
//Params.Init()
|
||||
|
||||
var err error
|
||||
// todo wait for proxyservice state changed to Healthy
|
||||
|
||||
err = node.masterClient.Init()
|
||||
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()
|
||||
err := node.waitForServiceReady(node.proxyServiceClient, "ProxyService")
|
||||
if err != nil {
|
||||
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 {
|
||||
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 {
|
||||
return err
|
||||
}
|
||||
Params.InsertChannelNames, err = node.dataServiceClient.GetInsertChannelNames()
|
||||
if err != nil {
|
||||
return err
|
||||
|
||||
// wait for dataservice state changed to Healthy
|
||||
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{
|
||||
ServiceName: "proxynode",
|
||||
Sampler: &config.SamplerConfig{
|
||||
|
@ -136,7 +213,7 @@ func (node *NodeImpl) Init() error {
|
|||
node.tsoAllocator = tsoAllocator
|
||||
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 {
|
||||
panic(err)
|
||||
}
|
||||
|
@ -162,23 +239,6 @@ func (node *NodeImpl) Init() 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)
|
||||
node.manipulationMsgStream.Start()
|
||||
node.queryMsgStream.Start()
|
||||
|
@ -206,23 +266,6 @@ func (node *NodeImpl) Stop() error {
|
|||
node.manipulationMsgStream.Close()
|
||||
node.queryMsgStream.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()
|
||||
|
||||
|
@ -253,3 +296,23 @@ func (node *NodeImpl) lastTick() Timestamp {
|
|||
func (node *NodeImpl) AddCloseCallback(callbacks ...func()) {
|
||||
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
|
||||
}
|
||||
|
|
|
@ -86,7 +86,7 @@ func startMaster(ctx context.Context) {
|
|||
|
||||
func startProxy(ctx context.Context) {
|
||||
|
||||
svr, err := CreateProxyNodeImpl(ctx)
|
||||
svr, err := NewProxyNodeImpl(ctx)
|
||||
proxyServer = svr
|
||||
if err != nil {
|
||||
log.Print("create proxynode failed", zap.Error(err))
|
||||
|
@ -130,7 +130,7 @@ func shutdown() {
|
|||
}
|
||||
|
||||
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!"
|
||||
assert.Nil(t, err, msg)
|
||||
return resp.Value
|
||||
|
@ -181,7 +181,7 @@ func createCollection(t *testing.T, name string) {
|
|||
CollectionName: name,
|
||||
Schema: schemaBytes,
|
||||
}
|
||||
resp, err := proxyServer.CreateCollection(ctx, req)
|
||||
resp, err := proxyServer.CreateCollection(req)
|
||||
assert.Nil(t, err)
|
||||
msg := "Create Collection " + name + " should succeed!"
|
||||
assert.Equal(t, resp.ErrorCode, commonpb.ErrorCode_SUCCESS, msg)
|
||||
|
@ -191,7 +191,7 @@ func dropCollection(t *testing.T, name string) {
|
|||
req := &milvuspb.DropCollectionRequest{
|
||||
CollectionName: name,
|
||||
}
|
||||
resp, err := proxyServer.DropCollection(ctx, req)
|
||||
resp, err := proxyServer.DropCollection(req)
|
||||
assert.Nil(t, err)
|
||||
msg := "Drop Collection " + name + " should succeed! err :" + resp.Reason
|
||||
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)
|
||||
msg := "Create Index for " + fieldName + " should succeed!"
|
||||
assert.Equal(t, resp.ErrorCode, commonpb.ErrorCode_SUCCESS, msg)
|
||||
|
@ -262,7 +262,7 @@ func TestProxy_DescribeCollection(t *testing.T) {
|
|||
createCollection(t, collectionName)
|
||||
has := hasCollection(t, collectionName)
|
||||
if has {
|
||||
resp, err := proxyServer.DescribeCollection(ctx, &milvuspb.DescribeCollectionRequest{CollectionName: collectionName})
|
||||
resp, err := proxyServer.DescribeCollection(&milvuspb.DescribeCollectionRequest{CollectionName: collectionName})
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
@ -288,7 +288,7 @@ func TestProxy_ShowCollections(t *testing.T) {
|
|||
createCollection(t, collectionName)
|
||||
has := hasCollection(t, collectionName)
|
||||
if has {
|
||||
resp, err := proxyServer.ShowCollections(ctx, &milvuspb.ShowCollectionRequest{})
|
||||
resp, err := proxyServer.ShowCollections(&milvuspb.ShowCollectionRequest{})
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
@ -321,7 +321,7 @@ func TestProxy_Insert(t *testing.T) {
|
|||
createCollection(t, collectionName)
|
||||
has := hasCollection(t, collectionName)
|
||||
if has {
|
||||
resp, err := proxyServer.Insert(ctx, req)
|
||||
resp, err := proxyServer.Insert(req)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
@ -397,7 +397,7 @@ func TestProxy_Search(t *testing.T) {
|
|||
if !has {
|
||||
createCollection(t, collectionName)
|
||||
}
|
||||
resp, err := proxyServer.Search(ctx, req)
|
||||
resp, err := proxyServer.Search(req)
|
||||
t.Logf("response of search collection %v: %v", i, resp)
|
||||
assert.Nil(t, err)
|
||||
dropCollection(t, collectionName)
|
||||
|
@ -458,7 +458,7 @@ func TestProxy_PartitionGRPC(t *testing.T) {
|
|||
PartitionName: tag,
|
||||
}
|
||||
|
||||
stb, err := proxyServer.HasPartition(ctx, preq)
|
||||
stb, err := proxyServer.HasPartition(preq)
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, stb.Status.ErrorCode, commonpb.ErrorCode_SUCCESS)
|
||||
assert.Equal(t, stb.Value, false)
|
||||
|
@ -467,11 +467,11 @@ func TestProxy_PartitionGRPC(t *testing.T) {
|
|||
CollectionName: collName,
|
||||
PartitionName: tag,
|
||||
}
|
||||
st, err := proxyServer.CreatePartition(ctx, cpreq)
|
||||
st, err := proxyServer.CreatePartition(cpreq)
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, st.ErrorCode, commonpb.ErrorCode_SUCCESS)
|
||||
|
||||
stb, err = proxyServer.HasPartition(ctx, preq)
|
||||
stb, err = proxyServer.HasPartition(preq)
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, stb.Status.ErrorCode, commonpb.ErrorCode_SUCCESS)
|
||||
assert.Equal(t, stb.Value, true)
|
||||
|
@ -480,7 +480,7 @@ func TestProxy_PartitionGRPC(t *testing.T) {
|
|||
//assert.Nil(t, err)
|
||||
//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.Equal(t, sts.Status.ErrorCode, commonpb.ErrorCode_SUCCESS)
|
||||
assert.True(t, len(sts.PartitionNames) >= 2)
|
||||
|
@ -490,7 +490,7 @@ func TestProxy_PartitionGRPC(t *testing.T) {
|
|||
CollectionName: collName,
|
||||
PartitionName: tag,
|
||||
}
|
||||
st, err = proxyServer.DropPartition(ctx, dpreq)
|
||||
st, err = proxyServer.DropPartition(dpreq)
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, st.ErrorCode, commonpb.ErrorCode_SUCCESS)
|
||||
}()
|
||||
|
@ -544,7 +544,7 @@ func TestProxy_DescribeIndex(t *testing.T) {
|
|||
CollectionName: collName,
|
||||
FieldName: fieldName,
|
||||
}
|
||||
resp, err := proxyServer.DescribeIndex(ctx, req)
|
||||
resp, err := proxyServer.DescribeIndex(req)
|
||||
assert.Nil(t, err)
|
||||
msg := "Describe Index for " + fieldName + "should successed!"
|
||||
assert.Equal(t, resp.Status.ErrorCode, commonpb.ErrorCode_SUCCESS, msg)
|
||||
|
@ -575,7 +575,7 @@ func TestProxy_GetIndexState(t *testing.T) {
|
|||
CollectionName: collName,
|
||||
FieldName: fieldName,
|
||||
}
|
||||
resp, err := proxyServer.GetIndexState(ctx, req)
|
||||
resp, err := proxyServer.GetIndexState(req)
|
||||
assert.Nil(t, err)
|
||||
msg := "Describe Index Progress for " + fieldName + "should succeed!"
|
||||
assert.Equal(t, resp.Status.ErrorCode, commonpb.ErrorCode_SUCCESS, msg)
|
||||
|
|
|
@ -9,7 +9,6 @@ import (
|
|||
|
||||
"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/msgstream"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
|
@ -17,7 +16,7 @@ import (
|
|||
|
||||
func insertRepackFunc(tsMsgs []msgstream.TsMsg,
|
||||
hashKeys [][]int32,
|
||||
segIDAssigner *allocator.SegIDAssigner,
|
||||
segIDAssigner *SegIDAssigner,
|
||||
together bool) (map[int32]*msgstream.MsgPack, error) {
|
||||
|
||||
result := make(map[int32]*msgstream.MsgPack)
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
package allocator
|
||||
package proxynode
|
||||
|
||||
import (
|
||||
"container/list"
|
||||
|
@ -8,6 +8,8 @@ import (
|
|||
"strconv"
|
||||
"time"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/allocator"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/errors"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
|
||||
|
@ -19,6 +21,20 @@ const (
|
|||
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 {
|
||||
segID UniqueID
|
||||
count uint32
|
||||
|
@ -110,30 +126,38 @@ type SegIDAssigner struct {
|
|||
segReqs []*datapb.SegIDRequest
|
||||
getTickFunc func() Timestamp
|
||||
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)
|
||||
sa := &SegIDAssigner{
|
||||
Allocator: Allocator{reqs: make(chan request, maxConcurrentRequests),
|
||||
ctx: ctx1,
|
||||
cancel: cancel,
|
||||
masterAddress: masterAddr,
|
||||
countPerRPC: SegCountPerRPC,
|
||||
Allocator: Allocator{
|
||||
Ctx: ctx1,
|
||||
CancelFunc: cancel,
|
||||
},
|
||||
assignInfos: make(map[string]*list.List),
|
||||
getTickFunc: getTickFunc,
|
||||
countPerRPC: SegCountPerRPC,
|
||||
serviceClient: client,
|
||||
assignInfos: make(map[string]*list.List),
|
||||
getTickFunc: getTickFunc,
|
||||
}
|
||||
sa.tChan = &ticker{
|
||||
updateInterval: time.Second,
|
||||
sa.TChan = &allocator.Ticker{
|
||||
UpdateInterval: time.Second,
|
||||
}
|
||||
sa.Allocator.syncFunc = sa.syncSegments
|
||||
sa.Allocator.processFunc = sa.processFunc
|
||||
sa.Allocator.checkSyncFunc = sa.checkSyncFunc
|
||||
sa.Allocator.pickCanDoFunc = sa.pickCanDoFunc
|
||||
sa.Allocator.SyncFunc = sa.syncSegments
|
||||
sa.Allocator.ProcessFunc = sa.processFunc
|
||||
sa.Allocator.CheckSyncFunc = sa.checkSyncFunc
|
||||
sa.Allocator.PickCanDoFunc = sa.pickCanDoFunc
|
||||
sa.Init()
|
||||
return sa, nil
|
||||
}
|
||||
|
||||
func (sa *SegIDAssigner) SetServiceClient(client DataServiceClient) {
|
||||
sa.serviceClient = client
|
||||
}
|
||||
|
||||
func (sa *SegIDAssigner) collectExpired() {
|
||||
ts := sa.getTickFunc()
|
||||
//now := time.Now()
|
||||
|
@ -149,12 +173,12 @@ func (sa *SegIDAssigner) collectExpired() {
|
|||
}
|
||||
|
||||
func (sa *SegIDAssigner) pickCanDoFunc() {
|
||||
if sa.toDoReqs == nil {
|
||||
if sa.ToDoReqs == nil {
|
||||
return
|
||||
}
|
||||
records := make(map[string]map[string]map[int32]uint32)
|
||||
newTodoReqs := sa.toDoReqs[0:0]
|
||||
for _, req := range sa.toDoReqs {
|
||||
newTodoReqs := sa.ToDoReqs[0:0]
|
||||
for _, req := range sa.ToDoReqs {
|
||||
segRequest := req.(*segRequest)
|
||||
colName := segRequest.colName
|
||||
partitionName := segRequest.partitionName
|
||||
|
@ -185,10 +209,10 @@ func (sa *SegIDAssigner) pickCanDoFunc() {
|
|||
})
|
||||
newTodoReqs = append(newTodoReqs, req)
|
||||
} 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 {
|
||||
|
@ -258,7 +282,7 @@ func (sa *SegIDAssigner) syncSegments() bool {
|
|||
return true
|
||||
}
|
||||
sa.reduceSegReqs()
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
|
||||
_, cancel := context.WithTimeout(context.Background(), 5*time.Second)
|
||||
defer cancel()
|
||||
req := &datapb.AssignSegIDRequest{
|
||||
NodeID: sa.PeerID,
|
||||
|
@ -267,7 +291,7 @@ func (sa *SegIDAssigner) syncSegments() bool {
|
|||
}
|
||||
|
||||
sa.segReqs = []*datapb.SegIDRequest{}
|
||||
resp, err := sa.masterClient.AssignSegmentID(ctx, req)
|
||||
resp, err := sa.serviceClient.AssignSegmentID(req)
|
||||
|
||||
if err != nil {
|
||||
log.Println("GRPC AssignSegmentID Failed", resp, err)
|
||||
|
@ -319,7 +343,7 @@ func (sa *SegIDAssigner) syncSegments() bool {
|
|||
return success
|
||||
}
|
||||
|
||||
func (sa *SegIDAssigner) processFunc(req request) error {
|
||||
func (sa *SegIDAssigner) processFunc(req allocator.Request) error {
|
||||
segRequest := req.(*segRequest)
|
||||
assign := sa.getAssign(segRequest.colName, segRequest.partitionName, segRequest.channelID)
|
||||
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) {
|
||||
req := &segRequest{
|
||||
baseRequest: baseRequest{done: make(chan error), valid: false},
|
||||
BaseRequest: allocator.BaseRequest{Done: make(chan error), Valid: false},
|
||||
colName: colName,
|
||||
partitionName: partitionName,
|
||||
channelID: channelID,
|
||||
count: count,
|
||||
timestamp: ts,
|
||||
}
|
||||
sa.reqs <- req
|
||||
sa.Reqs <- req
|
||||
req.Wait()
|
||||
|
||||
if !req.IsValid() {
|
|
@ -188,7 +188,7 @@ func (it *InsertTask) PostExecute() error {
|
|||
type CreateCollectionTask struct {
|
||||
Condition
|
||||
*milvuspb.CreateCollectionRequest
|
||||
masterClient MasterClientInterface
|
||||
masterClient MasterClient
|
||||
result *commonpb.Status
|
||||
ctx context.Context
|
||||
schema *schemapb.CollectionSchema
|
||||
|
@ -303,7 +303,7 @@ func (cct *CreateCollectionTask) PostExecute() error {
|
|||
type DropCollectionTask struct {
|
||||
Condition
|
||||
*milvuspb.DropCollectionRequest
|
||||
masterClient MasterClientInterface
|
||||
masterClient MasterClient
|
||||
result *commonpb.Status
|
||||
ctx context.Context
|
||||
}
|
||||
|
@ -645,7 +645,7 @@ func (st *SearchTask) PostExecute() error {
|
|||
type HasCollectionTask struct {
|
||||
Condition
|
||||
*milvuspb.HasCollectionRequest
|
||||
masterClient MasterClientInterface
|
||||
masterClient MasterClient
|
||||
result *milvuspb.BoolResponse
|
||||
ctx context.Context
|
||||
}
|
||||
|
@ -702,7 +702,7 @@ func (hct *HasCollectionTask) PostExecute() error {
|
|||
type DescribeCollectionTask struct {
|
||||
Condition
|
||||
*milvuspb.DescribeCollectionRequest
|
||||
masterClient MasterClientInterface
|
||||
masterClient MasterClient
|
||||
result *milvuspb.DescribeCollectionResponse
|
||||
ctx context.Context
|
||||
}
|
||||
|
@ -763,7 +763,7 @@ func (dct *DescribeCollectionTask) PostExecute() error {
|
|||
type ShowCollectionsTask struct {
|
||||
Condition
|
||||
*milvuspb.ShowCollectionRequest
|
||||
masterClient MasterClientInterface
|
||||
masterClient MasterClient
|
||||
result *milvuspb.ShowCollectionResponse
|
||||
ctx context.Context
|
||||
}
|
||||
|
@ -817,7 +817,7 @@ func (sct *ShowCollectionsTask) PostExecute() error {
|
|||
type CreatePartitionTask struct {
|
||||
Condition
|
||||
*milvuspb.CreatePartitionRequest
|
||||
masterClient MasterClientInterface
|
||||
masterClient MasterClient
|
||||
result *commonpb.Status
|
||||
ctx context.Context
|
||||
}
|
||||
|
@ -880,7 +880,7 @@ func (cpt *CreatePartitionTask) PostExecute() error {
|
|||
type DropPartitionTask struct {
|
||||
Condition
|
||||
*milvuspb.DropPartitionRequest
|
||||
masterClient MasterClientInterface
|
||||
masterClient MasterClient
|
||||
result *commonpb.Status
|
||||
ctx context.Context
|
||||
}
|
||||
|
@ -943,7 +943,7 @@ func (dpt *DropPartitionTask) PostExecute() error {
|
|||
type HasPartitionTask struct {
|
||||
Condition
|
||||
*milvuspb.HasPartitionRequest
|
||||
masterClient MasterClientInterface
|
||||
masterClient MasterClient
|
||||
result *milvuspb.BoolResponse
|
||||
ctx context.Context
|
||||
}
|
||||
|
@ -1059,7 +1059,7 @@ func (hpt *HasPartitionTask) PostExecute() error {
|
|||
type ShowPartitionsTask struct {
|
||||
Condition
|
||||
*milvuspb.ShowPartitionRequest
|
||||
masterClient MasterClientInterface
|
||||
masterClient MasterClient
|
||||
result *milvuspb.ShowPartitionResponse
|
||||
ctx context.Context
|
||||
}
|
||||
|
@ -1116,7 +1116,7 @@ func (spt *ShowPartitionsTask) PostExecute() error {
|
|||
type CreateIndexTask struct {
|
||||
Condition
|
||||
*milvuspb.CreateIndexRequest
|
||||
masterClient MasterClientInterface
|
||||
masterClient MasterClient
|
||||
result *commonpb.Status
|
||||
ctx context.Context
|
||||
}
|
||||
|
@ -1179,7 +1179,7 @@ func (cit *CreateIndexTask) PostExecute() error {
|
|||
type DescribeIndexTask struct {
|
||||
Condition
|
||||
*milvuspb.DescribeIndexRequest
|
||||
masterClient MasterClientInterface
|
||||
masterClient MasterClient
|
||||
result *milvuspb.DescribeIndexResponse
|
||||
ctx context.Context
|
||||
}
|
||||
|
|
|
@ -83,6 +83,7 @@ func (s *ServiceImpl) fillNodeInitParams() error {
|
|||
}
|
||||
|
||||
func (s *ServiceImpl) Init() error {
|
||||
|
||||
err := s.fillNodeInitParams()
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -102,7 +103,7 @@ func (s *ServiceImpl) Init() error {
|
|||
ttBarrier := newSoftTimeTickBarrier(s.ctx, nodeTimeTickMsgStream, []UniqueID{0}, 10)
|
||||
s.tick = newTimeTick(s.ctx, ttBarrier, serviceTimeTickMsgStream)
|
||||
|
||||
s.state.State.StateCode = internalpb2.StateCode_HEALTHY
|
||||
s.stateCode = internalpb2.StateCode_HEALTHY
|
||||
|
||||
return nil
|
||||
}
|
||||
|
@ -119,7 +120,24 @@ func (s *ServiceImpl) Stop() 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) {
|
||||
|
|
|
@ -17,8 +17,9 @@ type ServiceImpl struct {
|
|||
sched *TaskScheduler
|
||||
tick TimeTick
|
||||
nodeInfos *GlobalNodeInfoTable
|
||||
stateCode internalpb2.StateCode
|
||||
|
||||
state *internalpb2.ComponentStates
|
||||
//subStates *internalpb2.ComponentStates
|
||||
|
||||
dataServiceClient *dataservice.Client
|
||||
nodeStartParams []*commonpb.KeyValuePair
|
||||
|
@ -27,6 +28,23 @@ type ServiceImpl struct {
|
|||
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) {
|
||||
rand.Seed(time.Now().UnixNano())
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
|
@ -38,16 +56,17 @@ func CreateProxyService(ctx context.Context) (ProxyService, error) {
|
|||
s.allocator = NewNodeIDAllocator()
|
||||
s.sched = NewTaskScheduler(ctx1)
|
||||
s.nodeInfos = NewGlobalNodeInfoTable()
|
||||
|
||||
s.state = &internalpb2.ComponentStates{
|
||||
State: &internalpb2.ComponentInfo{
|
||||
NodeID: 0,
|
||||
Role: "proxyservice",
|
||||
StateCode: internalpb2.StateCode_INITIALIZING,
|
||||
},
|
||||
SubcomponentStates: nil,
|
||||
Status: &commonpb.Status{},
|
||||
}
|
||||
|
||||
s.stateCode = internalpb2.StateCode_ABNORMAL
|
||||
/*
|
||||
s.state = &internalpb2.ComponentStates{
|
||||
State: &internalpb2.ComponentInfo{
|
||||
NodeID: 0,
|
||||
Role: "proxyservice",
|
||||
StateCode: internalpb2.StateCode_INITIALIZING,
|
||||
},
|
||||
SubcomponentStates: nil,
|
||||
Status: &commonpb.Status{},
|
||||
}
|
||||
*/
|
||||
return s, nil
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
}
|
|
@ -17,10 +17,14 @@ echo $MILVUS_DIR
|
|||
go test -race -cover "${MILVUS_DIR}/kv/..." -failfast
|
||||
# TODO: remove to distributed
|
||||
#go test -race -cover "${MILVUS_DIR}/proxynode/..." -failfast
|
||||
go test -race -cover "${MILVUS_DIR}/writenode/..." -failfast
|
||||
go test -race -cover "${MILVUS_DIR}/datanode/..." -failfast
|
||||
go test -race -cover "${MILVUS_DIR}/master/..." -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}/writenode/..." -failfast
|
||||
#go test -race -cover "${MILVUS_DIR}/datanode/..." -failfast
|
||||
#go test -race -cover "${MILVUS_DIR}/master/..." -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}/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 "${MILVUS_DIR}/kv/..." "${MILVUS_DIR}/msgstream/..." "${MILVUS_DIR}/master/..." "${MILVUS_DIR}/querynode/..." -failfast
|
||||
|
|
Loading…
Reference in New Issue