mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-07 01:28:27 +08:00
Add log for allocator
Signed-off-by: zhenshan.cao <zhenshan.cao@zilliz.com>
This commit is contained in:
parent
a532444da7
commit
c6950eb7eb
@ -2,10 +2,13 @@ package allocator
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
"errors"
|
||||||
|
"fmt"
|
||||||
"sync"
|
"sync"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"errors"
|
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||||
|
"go.uber.org/zap"
|
||||||
)
|
)
|
||||||
|
|
||||||
const (
|
const (
|
||||||
@ -106,11 +109,13 @@ type Allocator struct {
|
|||||||
TChan TickerChan
|
TChan TickerChan
|
||||||
ForceSyncChan chan Request
|
ForceSyncChan chan Request
|
||||||
|
|
||||||
SyncFunc func() bool
|
SyncFunc func() (bool, error)
|
||||||
ProcessFunc func(req Request) error
|
ProcessFunc func(req Request) error
|
||||||
|
|
||||||
CheckSyncFunc func(timeout bool) bool
|
CheckSyncFunc func(timeout bool) bool
|
||||||
PickCanDoFunc func()
|
PickCanDoFunc func()
|
||||||
|
SyncErr error
|
||||||
|
Role string
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ta *Allocator) Start() error {
|
func (ta *Allocator) Start() error {
|
||||||
@ -183,7 +188,7 @@ func (ta *Allocator) pickCanDo() {
|
|||||||
func (ta *Allocator) sync(timeout bool) bool {
|
func (ta *Allocator) sync(timeout bool) bool {
|
||||||
if ta.SyncFunc == nil || ta.CheckSyncFunc == nil {
|
if ta.SyncFunc == nil || ta.CheckSyncFunc == nil {
|
||||||
ta.CanDoReqs = ta.ToDoReqs
|
ta.CanDoReqs = ta.ToDoReqs
|
||||||
ta.ToDoReqs = ta.ToDoReqs[0:0]
|
ta.ToDoReqs = nil
|
||||||
return true
|
return true
|
||||||
}
|
}
|
||||||
if !timeout && len(ta.ToDoReqs) == 0 {
|
if !timeout && len(ta.ToDoReqs) == 0 {
|
||||||
@ -193,7 +198,8 @@ func (ta *Allocator) sync(timeout bool) bool {
|
|||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
|
|
||||||
ret := ta.SyncFunc()
|
var ret bool
|
||||||
|
ret, ta.SyncErr = ta.SyncFunc()
|
||||||
|
|
||||||
if !timeout {
|
if !timeout {
|
||||||
ta.TChan.Reset()
|
ta.TChan.Reset()
|
||||||
@ -207,16 +213,28 @@ func (ta *Allocator) finishSyncRequest() {
|
|||||||
req.Notify(nil)
|
req.Notify(nil)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
ta.SyncReqs = ta.SyncReqs[0:0]
|
ta.SyncReqs = nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ta *Allocator) failRemainRequest() {
|
func (ta *Allocator) failRemainRequest() {
|
||||||
|
var err error
|
||||||
|
if ta.SyncErr != nil {
|
||||||
|
err = fmt.Errorf("%s failRemainRequest err:%w", ta.Role, ta.SyncErr)
|
||||||
|
} else {
|
||||||
|
errMsg := fmt.Sprintf("%s failRemainRequest unexpected error", ta.Role)
|
||||||
|
err = errors.New(errMsg)
|
||||||
|
}
|
||||||
|
if len(ta.ToDoReqs) > 0 {
|
||||||
|
log.Debug("Allocator has some reqs to fail",
|
||||||
|
zap.Any("Role", ta.Role),
|
||||||
|
zap.Any("reqLen", len(ta.ToDoReqs)))
|
||||||
|
}
|
||||||
for _, req := range ta.ToDoReqs {
|
for _, req := range ta.ToDoReqs {
|
||||||
if req != nil {
|
if req != nil {
|
||||||
req.Notify(errors.New("failed: unexpected error"))
|
req.Notify(err)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
ta.ToDoReqs = []Request{}
|
ta.ToDoReqs = nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ta *Allocator) finishRequest() {
|
func (ta *Allocator) finishRequest() {
|
||||||
@ -241,7 +259,8 @@ func (ta *Allocator) Close() {
|
|||||||
ta.CancelFunc()
|
ta.CancelFunc()
|
||||||
ta.wg.Wait()
|
ta.wg.Wait()
|
||||||
ta.TChan.Close()
|
ta.TChan.Close()
|
||||||
ta.revokeRequest(errors.New("closing"))
|
errMsg := fmt.Sprintf("%s is closing", ta.Role)
|
||||||
|
ta.revokeRequest(errors.New(errMsg))
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ta *Allocator) CleanCache() {
|
func (ta *Allocator) CleanCache() {
|
||||||
|
|||||||
@ -2,14 +2,16 @@ package allocator
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"log"
|
"fmt"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/zilliztech/milvus-distributed/internal/util/retry"
|
"go.uber.org/zap"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
|
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/util/retry"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -41,6 +43,7 @@ func NewIDAllocator(ctx context.Context, masterAddr string) (*IDAllocator, error
|
|||||||
Allocator: Allocator{
|
Allocator: Allocator{
|
||||||
Ctx: ctx1,
|
Ctx: ctx1,
|
||||||
CancelFunc: cancel,
|
CancelFunc: cancel,
|
||||||
|
Role: "IDAllocator",
|
||||||
},
|
},
|
||||||
countPerRPC: IDCountPerRPC,
|
countPerRPC: IDCountPerRPC,
|
||||||
masterAddress: masterAddr,
|
masterAddress: masterAddr,
|
||||||
@ -58,12 +61,11 @@ func (ia *IDAllocator) Start() error {
|
|||||||
connectMasterFn := func() error {
|
connectMasterFn := func() error {
|
||||||
return ia.connectMaster()
|
return ia.connectMaster()
|
||||||
}
|
}
|
||||||
err := retry.Retry(10, time.Millisecond*200, connectMasterFn)
|
err := retry.Retry(1000, time.Millisecond*200, connectMasterFn)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
panic("connect to master failed")
|
panic("connect to master failed")
|
||||||
}
|
}
|
||||||
ia.Allocator.Start()
|
return ia.Allocator.Start()
|
||||||
return nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ia *IDAllocator) connectMaster() error {
|
func (ia *IDAllocator) connectMaster() error {
|
||||||
@ -71,16 +73,31 @@ func (ia *IDAllocator) connectMaster() error {
|
|||||||
defer cancel()
|
defer cancel()
|
||||||
conn, err := grpc.DialContext(ctx, ia.masterAddress, grpc.WithInsecure(), grpc.WithBlock())
|
conn, err := grpc.DialContext(ctx, ia.masterAddress, grpc.WithInsecure(), grpc.WithBlock())
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Printf("Connect to master failed, error= %v", err)
|
log.Error("Connect to master failed", zap.Any("Role", ia.Role), zap.Error(err))
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
log.Printf("Connected to master, master_addr=%s", ia.masterAddress)
|
log.Debug("Connected to master", zap.Any("Role", ia.Role), zap.Any("masterAddress", ia.masterAddress))
|
||||||
ia.masterConn = conn
|
ia.masterConn = conn
|
||||||
ia.masterClient = masterpb.NewMasterServiceClient(conn)
|
ia.masterClient = masterpb.NewMasterServiceClient(conn)
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ia *IDAllocator) syncID() bool {
|
func (ia *IDAllocator) gatherReqIDCount() uint32 {
|
||||||
|
need := uint32(0)
|
||||||
|
for _, req := range ia.ToDoReqs {
|
||||||
|
tReq := req.(*IDRequest)
|
||||||
|
need += tReq.count
|
||||||
|
}
|
||||||
|
return need
|
||||||
|
}
|
||||||
|
|
||||||
|
func (ia *IDAllocator) syncID() (bool, error) {
|
||||||
|
|
||||||
|
need := ia.gatherReqIDCount()
|
||||||
|
if need < ia.countPerRPC {
|
||||||
|
need = ia.countPerRPC
|
||||||
|
}
|
||||||
|
|
||||||
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
|
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
|
||||||
req := &masterpb.AllocIDRequest{
|
req := &masterpb.AllocIDRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
@ -89,18 +106,17 @@ func (ia *IDAllocator) syncID() bool {
|
|||||||
Timestamp: 0,
|
Timestamp: 0,
|
||||||
SourceID: ia.PeerID,
|
SourceID: ia.PeerID,
|
||||||
},
|
},
|
||||||
Count: ia.countPerRPC,
|
Count: need,
|
||||||
}
|
}
|
||||||
resp, err := ia.masterClient.AllocID(ctx, req)
|
resp, err := ia.masterClient.AllocID(ctx, req)
|
||||||
|
|
||||||
cancel()
|
cancel()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Println("syncID Failed!!!!!")
|
return false, fmt.Errorf("syncID Failed:%w", err)
|
||||||
return false
|
|
||||||
}
|
}
|
||||||
ia.idStart = resp.GetID()
|
ia.idStart = resp.GetID()
|
||||||
ia.idEnd = ia.idStart + int64(resp.GetCount())
|
ia.idEnd = ia.idStart + int64(resp.GetCount())
|
||||||
return true
|
return true, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ia *IDAllocator) checkSyncFunc(timeout bool) bool {
|
func (ia *IDAllocator) checkSyncFunc(timeout bool) bool {
|
||||||
@ -122,6 +138,10 @@ func (ia *IDAllocator) pickCanDoFunc() {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
ia.ToDoReqs = ia.ToDoReqs[idx:]
|
ia.ToDoReqs = ia.ToDoReqs[idx:]
|
||||||
|
log.Debug("IDAllocator pickCanDoFunc",
|
||||||
|
zap.Any("need", need),
|
||||||
|
zap.Any("total", total),
|
||||||
|
zap.Any("remainReqCnt", len(ia.ToDoReqs)))
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ia *IDAllocator) processFunc(req Request) error {
|
func (ia *IDAllocator) processFunc(req Request) error {
|
||||||
|
|||||||
@ -3,9 +3,11 @@ package allocator
|
|||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"fmt"
|
"fmt"
|
||||||
"log"
|
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"go.uber.org/zap"
|
||||||
|
|
||||||
|
"github.com/zilliztech/milvus-distributed/internal/log"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
|
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
|
||||||
"github.com/zilliztech/milvus-distributed/internal/util/retry"
|
"github.com/zilliztech/milvus-distributed/internal/util/retry"
|
||||||
@ -38,6 +40,7 @@ func NewTimestampAllocator(ctx context.Context, masterAddr string) (*TimestampAl
|
|||||||
Allocator: Allocator{
|
Allocator: Allocator{
|
||||||
Ctx: ctx1,
|
Ctx: ctx1,
|
||||||
CancelFunc: cancel,
|
CancelFunc: cancel,
|
||||||
|
Role: "TimestampAllocator",
|
||||||
},
|
},
|
||||||
masterAddress: masterAddr,
|
masterAddress: masterAddr,
|
||||||
countPerRPC: tsCountPerRPC,
|
countPerRPC: tsCountPerRPC,
|
||||||
@ -57,7 +60,7 @@ func (ta *TimestampAllocator) Start() error {
|
|||||||
connectMasterFn := func() error {
|
connectMasterFn := func() error {
|
||||||
return ta.connectMaster()
|
return ta.connectMaster()
|
||||||
}
|
}
|
||||||
err := retry.Retry(10, time.Millisecond*200, connectMasterFn)
|
err := retry.Retry(1000, time.Millisecond*200, connectMasterFn)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
panic("Timestamp local allocator connect to master failed")
|
panic("Timestamp local allocator connect to master failed")
|
||||||
}
|
}
|
||||||
@ -70,10 +73,10 @@ func (ta *TimestampAllocator) connectMaster() error {
|
|||||||
defer cancel()
|
defer cancel()
|
||||||
conn, err := grpc.DialContext(ctx, ta.masterAddress, grpc.WithInsecure(), grpc.WithBlock())
|
conn, err := grpc.DialContext(ctx, ta.masterAddress, grpc.WithInsecure(), grpc.WithBlock())
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Printf("Connect to master failed, error= %v", err)
|
log.Error("TimestampAllocator Connect to master failed", zap.Error(err))
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
log.Printf("Connected to master, master_addr=%s", ta.masterAddress)
|
log.Debug("TimestampAllocator connected to master", zap.Any("masterAddress", ta.masterAddress))
|
||||||
ta.masterConn = conn
|
ta.masterConn = conn
|
||||||
ta.masterClient = masterpb.NewMasterServiceClient(conn)
|
ta.masterClient = masterpb.NewMasterServiceClient(conn)
|
||||||
return nil
|
return nil
|
||||||
@ -98,9 +101,26 @@ func (ta *TimestampAllocator) pickCanDoFunc() {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
ta.ToDoReqs = ta.ToDoReqs[idx:]
|
ta.ToDoReqs = ta.ToDoReqs[idx:]
|
||||||
|
log.Debug("TimestampAllocator pickCanDoFunc",
|
||||||
|
zap.Any("need", need),
|
||||||
|
zap.Any("total", total),
|
||||||
|
zap.Any("remainReqCnt", len(ta.ToDoReqs)))
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ta *TimestampAllocator) syncTs() bool {
|
func (ta *TimestampAllocator) gatherReqTsCount() uint32 {
|
||||||
|
need := uint32(0)
|
||||||
|
for _, req := range ta.ToDoReqs {
|
||||||
|
tReq := req.(*TSORequest)
|
||||||
|
need += tReq.count
|
||||||
|
}
|
||||||
|
return need
|
||||||
|
}
|
||||||
|
|
||||||
|
func (ta *TimestampAllocator) syncTs() (bool, error) {
|
||||||
|
need := ta.gatherReqTsCount()
|
||||||
|
if need < ta.countPerRPC {
|
||||||
|
need = ta.countPerRPC
|
||||||
|
}
|
||||||
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
|
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
|
||||||
req := &masterpb.AllocTimestampRequest{
|
req := &masterpb.AllocTimestampRequest{
|
||||||
Base: &commonpb.MsgBase{
|
Base: &commonpb.MsgBase{
|
||||||
@ -109,18 +129,18 @@ func (ta *TimestampAllocator) syncTs() bool {
|
|||||||
Timestamp: 0,
|
Timestamp: 0,
|
||||||
SourceID: ta.PeerID,
|
SourceID: ta.PeerID,
|
||||||
},
|
},
|
||||||
Count: ta.countPerRPC,
|
Count: need,
|
||||||
}
|
}
|
||||||
resp, err := ta.masterClient.AllocTimestamp(ctx, req)
|
|
||||||
|
|
||||||
cancel()
|
resp, err := ta.masterClient.AllocTimestamp(ctx, req)
|
||||||
|
defer cancel()
|
||||||
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Println("syncTimestamp Failed!!!!!")
|
return false, fmt.Errorf("syncTimestamp Failed:%w", err)
|
||||||
return false
|
|
||||||
}
|
}
|
||||||
ta.lastTsBegin = resp.GetTimestamp()
|
ta.lastTsBegin = resp.GetTimestamp()
|
||||||
ta.lastTsEnd = ta.lastTsBegin + uint64(resp.GetCount())
|
ta.lastTsEnd = ta.lastTsBegin + uint64(resp.GetCount())
|
||||||
return true
|
return true, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ta *TimestampAllocator) processFunc(req Request) error {
|
func (ta *TimestampAllocator) processFunc(req Request) error {
|
||||||
|
|||||||
@ -138,6 +138,7 @@ func NewSegIDAssigner(ctx context.Context, dataService types.DataService, getTic
|
|||||||
Allocator: Allocator{
|
Allocator: Allocator{
|
||||||
Ctx: ctx1,
|
Ctx: ctx1,
|
||||||
CancelFunc: cancel,
|
CancelFunc: cancel,
|
||||||
|
Role: "SegmentIDAllocator",
|
||||||
},
|
},
|
||||||
countPerRPC: SegCountPerRPC,
|
countPerRPC: SegCountPerRPC,
|
||||||
dataService: dataService,
|
dataService: dataService,
|
||||||
@ -275,9 +276,9 @@ func (sa *SegIDAssigner) reduceSegReqs() {
|
|||||||
sa.segReqs = newSegReqs
|
sa.segReqs = newSegReqs
|
||||||
}
|
}
|
||||||
|
|
||||||
func (sa *SegIDAssigner) syncSegments() bool {
|
func (sa *SegIDAssigner) syncSegments() (bool, error) {
|
||||||
if len(sa.segReqs) == 0 {
|
if len(sa.segReqs) == 0 {
|
||||||
return true
|
return true, nil
|
||||||
}
|
}
|
||||||
sa.reduceSegReqs()
|
sa.reduceSegReqs()
|
||||||
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
|
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
|
||||||
@ -292,8 +293,7 @@ func (sa *SegIDAssigner) syncSegments() bool {
|
|||||||
resp, err := sa.dataService.AssignSegmentID(ctx, req)
|
resp, err := sa.dataService.AssignSegmentID(ctx, req)
|
||||||
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Debug("proxynode", zap.String("GRPC AssignSegmentID Failed", err.Error()))
|
return false, fmt.Errorf("syncSegmentID Failed:%w", err)
|
||||||
return false
|
|
||||||
}
|
}
|
||||||
|
|
||||||
now := time.Now()
|
now := time.Now()
|
||||||
@ -331,7 +331,7 @@ func (sa *SegIDAssigner) syncSegments() bool {
|
|||||||
assign.lastInsertTime = now
|
assign.lastInsertTime = now
|
||||||
success = true
|
success = true
|
||||||
}
|
}
|
||||||
return success
|
return success, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (sa *SegIDAssigner) processFunc(req allocator.Request) error {
|
func (sa *SegIDAssigner) processFunc(req allocator.Request) error {
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user