2021-11-10 23:56:01 +08:00
|
|
|
// Licensed to the LF AI & Data foundation under one
|
|
|
|
// or more contributor license agreements. See the NOTICE file
|
|
|
|
// distributed with this work for additional information
|
|
|
|
// regarding copyright ownership. The ASF licenses this file
|
|
|
|
// to you under the Apache License, Version 2.0 (the
|
|
|
|
// "License"); you may not use this file except in compliance
|
2021-04-19 10:09:43 +08:00
|
|
|
// with the License. You may obtain a copy of the License at
|
|
|
|
//
|
2021-11-10 23:56:01 +08:00
|
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
2021-04-19 10:09:43 +08:00
|
|
|
//
|
2021-11-10 23:56:01 +08:00
|
|
|
// Unless required by applicable law or agreed to in writing, software
|
|
|
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
|
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
|
|
// See the License for the specific language governing permissions and
|
|
|
|
// limitations under the License.
|
2021-04-19 10:09:43 +08:00
|
|
|
|
2021-06-22 14:40:07 +08:00
|
|
|
package proxy
|
2021-01-22 09:36:18 +08:00
|
|
|
|
|
|
|
import (
|
|
|
|
"context"
|
2022-09-16 09:56:47 +08:00
|
|
|
"fmt"
|
2021-01-22 09:36:18 +08:00
|
|
|
"math/rand"
|
2022-03-17 17:17:22 +08:00
|
|
|
"os"
|
2022-03-02 16:23:55 +08:00
|
|
|
"strconv"
|
2021-01-22 09:36:18 +08:00
|
|
|
"sync"
|
2021-12-10 22:09:17 +08:00
|
|
|
"syscall"
|
2021-01-22 09:36:18 +08:00
|
|
|
"time"
|
|
|
|
|
2023-02-26 11:31:49 +08:00
|
|
|
"github.com/cockroachdb/errors"
|
2022-04-07 22:05:32 +08:00
|
|
|
clientv3 "go.etcd.io/etcd/client/v3"
|
2023-10-09 10:09:33 +08:00
|
|
|
"go.uber.org/atomic"
|
2022-04-07 22:05:32 +08:00
|
|
|
"go.uber.org/zap"
|
|
|
|
|
2023-06-09 01:28:37 +08:00
|
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
2023-09-21 09:45:27 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/allocator"
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
2022-11-10 17:09:06 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proxy/accesslog"
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/types"
|
2023-09-21 09:45:27 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/dependency"
|
|
|
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
2023-04-06 19:14:32 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/log"
|
|
|
|
"github.com/milvus-io/milvus/pkg/metrics"
|
2023-10-20 14:26:09 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/mq/msgstream"
|
2023-04-06 19:14:32 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/util/commonpbutil"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/logutil"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/ratelimitutil"
|
2023-10-20 14:26:09 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/util/resource"
|
2023-09-21 09:45:27 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/util/tsoutil"
|
2023-04-06 19:14:32 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
2021-01-22 09:36:18 +08:00
|
|
|
)
|
|
|
|
|
2021-10-28 23:56:40 +08:00
|
|
|
// UniqueID is alias of typeutil.UniqueID
|
2021-01-22 09:36:18 +08:00
|
|
|
type UniqueID = typeutil.UniqueID
|
2021-10-28 23:56:40 +08:00
|
|
|
|
|
|
|
// Timestamp is alias of typeutil.Timestamp
|
2021-01-22 09:36:18 +08:00
|
|
|
type Timestamp = typeutil.Timestamp
|
|
|
|
|
2021-12-30 10:29:47 +08:00
|
|
|
// const sendTimeTickMsgInterval = 200 * time.Millisecond
|
|
|
|
// const channelMgrTickerInterval = 100 * time.Millisecond
|
2021-06-15 10:19:38 +08:00
|
|
|
|
2021-10-04 17:28:30 +08:00
|
|
|
// make sure Proxy implements types.Proxy
|
|
|
|
var _ types.Proxy = (*Proxy)(nil)
|
|
|
|
|
2022-11-04 14:25:38 +08:00
|
|
|
var Params *paramtable.ComponentParam = paramtable.Get()
|
2021-12-23 18:39:11 +08:00
|
|
|
|
2022-09-16 09:56:47 +08:00
|
|
|
// rateCol is global rateCollector in Proxy.
|
|
|
|
var rateCol *ratelimitutil.RateCollector
|
|
|
|
|
2021-12-15 21:11:35 +08:00
|
|
|
// Proxy of milvus
|
2021-06-22 14:40:07 +08:00
|
|
|
type Proxy struct {
|
2021-01-22 09:36:18 +08:00
|
|
|
ctx context.Context
|
2023-07-28 10:11:08 +08:00
|
|
|
cancel context.CancelFunc
|
2021-01-22 09:36:18 +08:00
|
|
|
wg sync.WaitGroup
|
|
|
|
|
2021-03-12 14:22:09 +08:00
|
|
|
initParams *internalpb.InitParams
|
2021-01-28 20:51:44 +08:00
|
|
|
ip string
|
|
|
|
port int
|
2021-01-22 09:36:18 +08:00
|
|
|
|
2023-10-09 10:09:33 +08:00
|
|
|
stateCode atomic.Int32
|
2021-01-29 09:27:26 +08:00
|
|
|
|
2021-12-29 14:35:21 +08:00
|
|
|
etcdCli *clientv3.Client
|
2022-11-04 14:25:38 +08:00
|
|
|
address string
|
2023-09-26 09:57:25 +08:00
|
|
|
rootCoord types.RootCoordClient
|
|
|
|
dataCoord types.DataCoordClient
|
|
|
|
queryCoord types.QueryCoordClient
|
2021-01-28 20:51:44 +08:00
|
|
|
|
2022-09-16 09:56:47 +08:00
|
|
|
multiRateLimiter *MultiRateLimiter
|
|
|
|
|
2021-05-27 17:09:50 +08:00
|
|
|
chMgr channelsMgr
|
|
|
|
|
2023-10-20 14:26:09 +08:00
|
|
|
replicateMsgStream msgstream.MsgStream
|
|
|
|
|
2021-09-06 20:49:04 +08:00
|
|
|
sched *taskScheduler
|
2021-01-22 09:36:18 +08:00
|
|
|
|
2021-05-29 09:55:29 +08:00
|
|
|
chTicker channelsTimeTicker
|
|
|
|
|
2022-10-09 10:06:58 +08:00
|
|
|
rowIDAllocator *allocator.IDAllocator
|
|
|
|
tsoAllocator *timestampAllocator
|
|
|
|
segAssigner *segIDAssigner
|
2021-01-22 09:36:18 +08:00
|
|
|
|
2021-09-03 17:15:26 +08:00
|
|
|
metricsCacheManager *metricsinfo.MetricsCacheManager
|
|
|
|
|
2022-06-02 12:16:03 +08:00
|
|
|
session *sessionutil.Session
|
2023-06-13 10:20:37 +08:00
|
|
|
shardMgr shardClientMgr
|
2021-05-21 19:28:52 +08:00
|
|
|
|
2022-04-07 22:05:32 +08:00
|
|
|
factory dependency.Factory
|
2021-01-22 09:36:18 +08:00
|
|
|
|
2022-04-20 16:15:41 +08:00
|
|
|
searchResultCh chan *internalpb.SearchResults
|
2022-01-17 14:41:35 +08:00
|
|
|
|
2021-01-22 09:36:18 +08:00
|
|
|
// Add callback functions at different stages
|
|
|
|
startCallbacks []func()
|
|
|
|
closeCallbacks []func()
|
2023-06-13 10:20:37 +08:00
|
|
|
|
|
|
|
// for load balance in replicas
|
|
|
|
lbPolicy LBPolicy
|
2023-10-20 14:26:09 +08:00
|
|
|
|
|
|
|
// resource manager
|
|
|
|
resourceManager resource.Manager
|
|
|
|
replicateStreamManager *ReplicateStreamManager
|
2021-01-22 09:36:18 +08:00
|
|
|
}
|
|
|
|
|
2021-10-07 20:24:40 +08:00
|
|
|
// NewProxy returns a Proxy struct.
|
2022-04-07 22:05:32 +08:00
|
|
|
func NewProxy(ctx context.Context, factory dependency.Factory) (*Proxy, error) {
|
2021-01-22 09:36:18 +08:00
|
|
|
rand.Seed(time.Now().UnixNano())
|
|
|
|
ctx1, cancel := context.WithCancel(ctx)
|
2022-01-17 14:41:35 +08:00
|
|
|
n := 1024 // better to be configurable
|
2023-06-13 10:20:37 +08:00
|
|
|
mgr := newShardClientMgr()
|
2023-06-27 09:52:44 +08:00
|
|
|
lbPolicy := NewLBPolicyImpl(mgr)
|
|
|
|
lbPolicy.Start(ctx)
|
2023-10-20 14:26:09 +08:00
|
|
|
resourceManager := resource.NewManager(10*time.Second, 20*time.Second, make(map[string]time.Duration))
|
|
|
|
replicateStreamManager := NewReplicateStreamManager(ctx, factory, resourceManager)
|
2021-06-22 14:40:07 +08:00
|
|
|
node := &Proxy{
|
2023-10-20 14:26:09 +08:00
|
|
|
ctx: ctx1,
|
|
|
|
cancel: cancel,
|
|
|
|
factory: factory,
|
|
|
|
searchResultCh: make(chan *internalpb.SearchResults, n),
|
|
|
|
shardMgr: mgr,
|
|
|
|
multiRateLimiter: NewMultiRateLimiter(),
|
|
|
|
lbPolicy: lbPolicy,
|
|
|
|
resourceManager: resourceManager,
|
|
|
|
replicateStreamManager: replicateStreamManager,
|
2021-01-22 09:36:18 +08:00
|
|
|
}
|
2022-10-10 15:55:22 +08:00
|
|
|
node.UpdateStateCode(commonpb.StateCode_Abnormal)
|
2021-12-17 19:07:39 +08:00
|
|
|
logutil.Logger(ctx).Debug("create a new Proxy instance", zap.Any("state", node.stateCode.Load()))
|
2021-01-22 09:36:18 +08:00
|
|
|
return node, nil
|
|
|
|
}
|
|
|
|
|
2023-10-09 10:09:33 +08:00
|
|
|
// UpdateStateCode updates the state code of Proxy.
|
|
|
|
func (node *Proxy) UpdateStateCode(code commonpb.StateCode) {
|
|
|
|
node.stateCode.Store(int32(code))
|
|
|
|
}
|
|
|
|
|
|
|
|
func (node *Proxy) GetStateCode() commonpb.StateCode {
|
|
|
|
return commonpb.StateCode(node.stateCode.Load())
|
|
|
|
}
|
|
|
|
|
2021-12-06 19:15:48 +08:00
|
|
|
// Register registers proxy at etcd
|
2021-06-22 14:40:07 +08:00
|
|
|
func (node *Proxy) Register() error {
|
2021-12-15 11:47:10 +08:00
|
|
|
node.session.Register()
|
2023-06-26 17:52:44 +08:00
|
|
|
metrics.NumNodes.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), typeutil.ProxyRole).Inc()
|
|
|
|
log.Info("Proxy Register Finished")
|
2023-04-12 20:12:28 +08:00
|
|
|
node.session.LivenessCheck(node.ctx, func() {
|
2021-10-30 10:24:38 +08:00
|
|
|
log.Error("Proxy disconnected from etcd, process will exit", zap.Int64("Server Id", node.session.ServerID))
|
|
|
|
if err := node.Stop(); err != nil {
|
|
|
|
log.Fatal("failed to stop server", zap.Error(err))
|
|
|
|
}
|
2023-06-26 17:52:44 +08:00
|
|
|
metrics.NumNodes.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), typeutil.ProxyRole).Dec()
|
2021-12-29 14:35:21 +08:00
|
|
|
if node.session.TriggerKill {
|
2022-03-17 17:17:22 +08:00
|
|
|
if p, err := os.FindProcess(os.Getpid()); err == nil {
|
|
|
|
p.Signal(syscall.SIGINT)
|
|
|
|
}
|
2021-12-29 14:35:21 +08:00
|
|
|
}
|
2021-10-27 21:58:33 +08:00
|
|
|
})
|
2021-09-27 17:37:57 +08:00
|
|
|
// TODO Reset the logger
|
2023-09-21 09:45:27 +08:00
|
|
|
// Params.initLogCfg()
|
2021-05-25 15:06:05 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-12-29 14:29:37 +08:00
|
|
|
// initSession initialize the session of Proxy.
|
2021-12-15 11:47:10 +08:00
|
|
|
func (node *Proxy) initSession() error {
|
2023-10-27 07:36:12 +08:00
|
|
|
node.session = sessionutil.NewSession(node.ctx)
|
2021-12-15 11:47:10 +08:00
|
|
|
if node.session == nil {
|
|
|
|
return errors.New("new session failed, maybe etcd cannot be connected")
|
|
|
|
}
|
2022-11-04 14:25:38 +08:00
|
|
|
node.session.Init(typeutil.ProxyRole, node.address, false, true)
|
2023-10-16 10:24:10 +08:00
|
|
|
sessionutil.SaveServerInfo(typeutil.ProxyRole, node.session.ServerID)
|
2021-12-15 11:47:10 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2022-09-16 09:56:47 +08:00
|
|
|
// initRateCollector creates and starts rateCollector in Proxy.
|
|
|
|
func (node *Proxy) initRateCollector() error {
|
|
|
|
var err error
|
|
|
|
rateCol, err = ratelimitutil.NewRateCollector(ratelimitutil.DefaultWindow, ratelimitutil.DefaultGranularity)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
rateCol.Register(internalpb.RateType_DMLInsert.String())
|
2023-07-11 11:20:34 +08:00
|
|
|
rateCol.Register(internalpb.RateType_DMLUpsert.String())
|
2022-09-16 09:56:47 +08:00
|
|
|
rateCol.Register(internalpb.RateType_DMLDelete.String())
|
|
|
|
// TODO: add bulkLoad rate
|
|
|
|
rateCol.Register(internalpb.RateType_DQLSearch.String())
|
|
|
|
rateCol.Register(internalpb.RateType_DQLQuery.String())
|
2022-10-13 14:57:24 +08:00
|
|
|
rateCol.Register(metricsinfo.ReadResultThroughput)
|
2022-09-16 09:56:47 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-10-14 19:14:35 +08:00
|
|
|
// Init initialize proxy.
|
2021-06-22 14:40:07 +08:00
|
|
|
func (node *Proxy) Init() error {
|
2022-04-14 20:01:34 +08:00
|
|
|
log.Info("init session for Proxy")
|
2021-12-25 16:10:17 +08:00
|
|
|
if err := node.initSession(); err != nil {
|
|
|
|
log.Warn("failed to init Proxy's session", zap.Error(err))
|
2021-12-15 11:47:10 +08:00
|
|
|
return err
|
|
|
|
}
|
2022-04-14 20:01:34 +08:00
|
|
|
log.Info("init session for Proxy done")
|
2021-01-29 09:27:26 +08:00
|
|
|
|
2022-11-04 14:25:38 +08:00
|
|
|
node.factory.Init(Params)
|
2021-02-08 14:30:54 +08:00
|
|
|
|
2022-11-10 17:09:06 +08:00
|
|
|
accesslog.SetupAccseeLog(&Params.ProxyCfg.AccessLog, &Params.MinioCfg)
|
|
|
|
log.Debug("init access log for Proxy done")
|
|
|
|
|
2022-09-16 09:56:47 +08:00
|
|
|
err := node.initRateCollector()
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2022-11-04 14:25:38 +08:00
|
|
|
log.Info("Proxy init rateCollector done", zap.Int64("nodeID", paramtable.GetNodeID()))
|
2022-09-16 09:56:47 +08:00
|
|
|
|
2022-11-04 14:25:38 +08:00
|
|
|
idAllocator, err := allocator.NewIDAllocator(node.ctx, node.rootCoord, paramtable.GetNodeID())
|
2021-01-22 09:36:18 +08:00
|
|
|
if err != nil {
|
2021-12-25 16:10:17 +08:00
|
|
|
log.Warn("failed to create id allocator",
|
2023-06-19 13:28:41 +08:00
|
|
|
zap.String("role", typeutil.ProxyRole), zap.Int64("ProxyID", paramtable.GetNodeID()),
|
|
|
|
zap.Error(err))
|
2021-01-22 09:36:18 +08:00
|
|
|
return err
|
|
|
|
}
|
2022-10-09 10:06:58 +08:00
|
|
|
node.rowIDAllocator = idAllocator
|
2022-11-04 14:25:38 +08:00
|
|
|
log.Debug("create id allocator done", zap.String("role", typeutil.ProxyRole), zap.Int64("ProxyID", paramtable.GetNodeID()))
|
2021-01-22 09:36:18 +08:00
|
|
|
|
2023-03-09 18:51:52 +08:00
|
|
|
tsoAllocator, err := newTimestampAllocator(node.rootCoord, paramtable.GetNodeID())
|
2021-01-22 09:36:18 +08:00
|
|
|
if err != nil {
|
2021-12-25 16:10:17 +08:00
|
|
|
log.Warn("failed to create timestamp allocator",
|
2023-06-19 13:28:41 +08:00
|
|
|
zap.String("role", typeutil.ProxyRole), zap.Int64("ProxyID", paramtable.GetNodeID()),
|
|
|
|
zap.Error(err))
|
2021-01-22 09:36:18 +08:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
node.tsoAllocator = tsoAllocator
|
2022-11-04 14:25:38 +08:00
|
|
|
log.Debug("create timestamp allocator done", zap.String("role", typeutil.ProxyRole), zap.Int64("ProxyID", paramtable.GetNodeID()))
|
2021-01-22 09:36:18 +08:00
|
|
|
|
2021-09-28 23:06:15 +08:00
|
|
|
segAssigner, err := newSegIDAssigner(node.ctx, node.dataCoord, node.lastTick)
|
2021-01-22 09:36:18 +08:00
|
|
|
if err != nil {
|
2021-12-25 16:10:17 +08:00
|
|
|
log.Warn("failed to create segment id assigner",
|
2023-06-19 13:28:41 +08:00
|
|
|
zap.String("role", typeutil.ProxyRole), zap.Int64("ProxyID", paramtable.GetNodeID()),
|
|
|
|
zap.Error(err))
|
2021-12-25 16:10:17 +08:00
|
|
|
return err
|
2021-01-22 09:36:18 +08:00
|
|
|
}
|
|
|
|
node.segAssigner = segAssigner
|
2022-11-04 14:25:38 +08:00
|
|
|
node.segAssigner.PeerID = paramtable.GetNodeID()
|
|
|
|
log.Debug("create segment id assigner done", zap.String("role", typeutil.ProxyRole), zap.Int64("ProxyID", paramtable.GetNodeID()))
|
2021-01-22 09:36:18 +08:00
|
|
|
|
2021-09-13 17:12:19 +08:00
|
|
|
dmlChannelsFunc := getDmlChannelsFunc(node.ctx, node.rootCoord)
|
2022-06-02 15:34:04 +08:00
|
|
|
chMgr := newChannelsMgrImpl(dmlChannelsFunc, defaultInsertRepackFunc, node.factory)
|
2021-05-27 17:09:50 +08:00
|
|
|
node.chMgr = chMgr
|
2021-12-25 16:10:17 +08:00
|
|
|
log.Debug("create channels manager done", zap.String("role", typeutil.ProxyRole))
|
2021-05-27 17:09:50 +08:00
|
|
|
|
2023-10-20 14:26:09 +08:00
|
|
|
replicateMsgChannel := Params.CommonCfg.ReplicateMsgChannel.GetValue()
|
|
|
|
node.replicateMsgStream, err = node.factory.NewMsgStream(node.ctx)
|
|
|
|
if err != nil {
|
|
|
|
log.Warn("failed to create replicate msg stream",
|
|
|
|
zap.String("role", typeutil.ProxyRole), zap.Int64("ProxyID", paramtable.GetNodeID()),
|
|
|
|
zap.Error(err))
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
node.replicateMsgStream.EnableProduce(true)
|
|
|
|
node.replicateMsgStream.AsProducer([]string{replicateMsgChannel})
|
|
|
|
|
2022-10-09 10:06:58 +08:00
|
|
|
node.sched, err = newTaskScheduler(node.ctx, node.tsoAllocator, node.factory)
|
2021-01-22 09:36:18 +08:00
|
|
|
if err != nil {
|
2023-06-19 13:28:41 +08:00
|
|
|
log.Warn("failed to create task scheduler", zap.String("role", typeutil.ProxyRole), zap.Error(err))
|
2021-01-22 09:36:18 +08:00
|
|
|
return err
|
|
|
|
}
|
2021-12-25 16:10:17 +08:00
|
|
|
log.Debug("create task scheduler done", zap.String("role", typeutil.ProxyRole))
|
2021-01-22 09:36:18 +08:00
|
|
|
|
2022-12-07 18:01:19 +08:00
|
|
|
syncTimeTickInterval := Params.ProxyCfg.TimeTickInterval.GetAsDuration(time.Millisecond) / 2
|
|
|
|
node.chTicker = newChannelsTimeTicker(node.ctx, Params.ProxyCfg.TimeTickInterval.GetAsDuration(time.Millisecond)/2, []string{}, node.sched.getPChanStatistics, tsoAllocator)
|
2023-06-19 13:28:41 +08:00
|
|
|
log.Debug("create channels time ticker done", zap.String("role", typeutil.ProxyRole), zap.Duration("syncTimeTickInterval", syncTimeTickInterval))
|
2021-05-29 09:55:29 +08:00
|
|
|
|
2021-09-03 17:15:26 +08:00
|
|
|
node.metricsCacheManager = metricsinfo.NewMetricsCacheManager()
|
2021-12-25 16:10:17 +08:00
|
|
|
log.Debug("create metrics cache manager done", zap.String("role", typeutil.ProxyRole))
|
|
|
|
|
2022-08-04 11:04:34 +08:00
|
|
|
if err := InitMetaCache(node.ctx, node.rootCoord, node.queryCoord, node.shardMgr); err != nil {
|
2023-06-19 13:28:41 +08:00
|
|
|
log.Warn("failed to init meta cache", zap.String("role", typeutil.ProxyRole), zap.Error(err))
|
2021-12-25 16:10:17 +08:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
log.Debug("init meta cache done", zap.String("role", typeutil.ProxyRole))
|
2021-09-03 17:15:26 +08:00
|
|
|
|
2023-11-27 16:30:26 +08:00
|
|
|
log.Info("init proxy done", zap.Int64("nodeID", paramtable.GetNodeID()), zap.String("Address", node.address))
|
2021-01-22 09:36:18 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-12-27 18:58:20 +08:00
|
|
|
// sendChannelsTimeTickLoop starts a goroutine that synchronizes the time tick information.
|
2021-06-22 14:40:07 +08:00
|
|
|
func (node *Proxy) sendChannelsTimeTickLoop() {
|
2021-05-31 17:28:31 +08:00
|
|
|
node.wg.Add(1)
|
|
|
|
go func() {
|
|
|
|
defer node.wg.Done()
|
|
|
|
|
2023-02-23 18:59:45 +08:00
|
|
|
ticker := time.NewTicker(Params.ProxyCfg.TimeTickInterval.GetAsDuration(time.Millisecond))
|
|
|
|
defer ticker.Stop()
|
2021-05-31 17:28:31 +08:00
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-node.ctx.Done():
|
2021-12-30 10:29:47 +08:00
|
|
|
log.Info("send channels time tick loop exit")
|
2021-05-31 17:28:31 +08:00
|
|
|
return
|
2023-02-23 18:59:45 +08:00
|
|
|
case <-ticker.C:
|
2023-10-20 14:26:09 +08:00
|
|
|
if !Params.CommonCfg.TTMsgEnabled.GetAsBool() {
|
|
|
|
continue
|
|
|
|
}
|
2021-11-05 09:14:02 +08:00
|
|
|
stats, ts, err := node.chTicker.getMinTsStatistics()
|
2021-06-18 15:14:08 +08:00
|
|
|
if err != nil {
|
2021-11-05 09:14:02 +08:00
|
|
|
log.Warn("sendChannelsTimeTickLoop.getMinTsStatistics", zap.Error(err))
|
2021-06-18 15:14:08 +08:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
2021-11-05 09:14:02 +08:00
|
|
|
if ts == 0 {
|
|
|
|
log.Warn("sendChannelsTimeTickLoop.getMinTsStatistics default timestamp equal 0")
|
2021-05-31 17:28:31 +08:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
2021-06-08 19:25:37 +08:00
|
|
|
channels := make([]pChan, 0, len(stats))
|
|
|
|
tss := make([]Timestamp, 0, len(stats))
|
|
|
|
|
2021-06-18 15:14:08 +08:00
|
|
|
maxTs := ts
|
2021-06-08 19:25:37 +08:00
|
|
|
for channel, ts := range stats {
|
|
|
|
channels = append(channels, channel)
|
|
|
|
tss = append(tss, ts)
|
2021-06-18 15:14:08 +08:00
|
|
|
if ts > maxTs {
|
|
|
|
maxTs = ts
|
|
|
|
}
|
2021-06-08 19:25:37 +08:00
|
|
|
}
|
2021-06-18 15:14:08 +08:00
|
|
|
|
2021-05-31 17:28:31 +08:00
|
|
|
req := &internalpb.ChannelTimeTickMsg{
|
2022-10-19 10:01:26 +08:00
|
|
|
Base: commonpbutil.NewMsgBase(
|
2023-11-24 15:02:39 +08:00
|
|
|
commonpbutil.WithMsgType(commonpb.MsgType_TimeTick),
|
2022-10-19 10:01:26 +08:00
|
|
|
commonpbutil.WithSourceID(node.session.ServerID),
|
|
|
|
),
|
2021-06-18 15:14:08 +08:00
|
|
|
ChannelNames: channels,
|
|
|
|
Timestamps: tss,
|
|
|
|
DefaultTimestamp: maxTs,
|
2021-05-31 17:28:31 +08:00
|
|
|
}
|
2023-04-06 15:28:33 +08:00
|
|
|
|
|
|
|
func() {
|
|
|
|
// we should pay more attention to the max lag.
|
|
|
|
minTs := maxTs
|
|
|
|
minTsOfChannel := "default"
|
|
|
|
|
|
|
|
// find the min ts and the related channel.
|
|
|
|
for channel, ts := range stats {
|
|
|
|
if ts < minTs {
|
|
|
|
minTs = ts
|
|
|
|
minTsOfChannel = channel
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
sub := tsoutil.SubByNow(minTs)
|
|
|
|
metrics.ProxySyncTimeTickLag.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), minTsOfChannel).Set(float64(sub))
|
|
|
|
}()
|
|
|
|
|
2021-06-21 17:28:03 +08:00
|
|
|
status, err := node.rootCoord.UpdateChannelTimeTick(node.ctx, req)
|
2021-05-31 17:28:31 +08:00
|
|
|
if err != nil {
|
|
|
|
log.Warn("sendChannelsTimeTickLoop.UpdateChannelTimeTick", zap.Error(err))
|
|
|
|
continue
|
|
|
|
}
|
2023-09-15 10:09:21 +08:00
|
|
|
if status.GetErrorCode() != 0 {
|
2021-05-31 17:28:31 +08:00
|
|
|
log.Warn("sendChannelsTimeTickLoop.UpdateChannelTimeTick",
|
|
|
|
zap.Any("ErrorCode", status.ErrorCode),
|
|
|
|
zap.Any("Reason", status.Reason))
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}()
|
|
|
|
}
|
|
|
|
|
2021-10-16 22:06:57 +08:00
|
|
|
// Start starts a proxy node.
|
2021-06-22 14:40:07 +08:00
|
|
|
func (node *Proxy) Start() error {
|
2021-09-09 10:06:29 +08:00
|
|
|
if err := node.sched.Start(); err != nil {
|
2023-06-19 13:28:41 +08:00
|
|
|
log.Warn("failed to start task scheduler", zap.String("role", typeutil.ProxyRole), zap.Error(err))
|
2021-09-09 10:06:29 +08:00
|
|
|
return err
|
|
|
|
}
|
2021-12-25 16:10:17 +08:00
|
|
|
log.Debug("start task scheduler done", zap.String("role", typeutil.ProxyRole))
|
2021-01-29 17:29:31 +08:00
|
|
|
|
2022-10-09 10:06:58 +08:00
|
|
|
if err := node.rowIDAllocator.Start(); err != nil {
|
2023-06-19 13:28:41 +08:00
|
|
|
log.Warn("failed to start id allocator", zap.String("role", typeutil.ProxyRole), zap.Error(err))
|
2021-09-09 10:06:29 +08:00
|
|
|
return err
|
|
|
|
}
|
2021-12-25 16:10:17 +08:00
|
|
|
log.Debug("start id allocator done", zap.String("role", typeutil.ProxyRole))
|
2021-01-29 17:29:31 +08:00
|
|
|
|
2021-09-09 10:06:29 +08:00
|
|
|
if err := node.segAssigner.Start(); err != nil {
|
2023-06-19 13:28:41 +08:00
|
|
|
log.Warn("failed to start segment id assigner", zap.String("role", typeutil.ProxyRole), zap.Error(err))
|
2021-09-09 10:06:29 +08:00
|
|
|
return err
|
|
|
|
}
|
2021-12-25 16:10:17 +08:00
|
|
|
log.Debug("start segment id assigner done", zap.String("role", typeutil.ProxyRole))
|
2021-01-29 17:29:31 +08:00
|
|
|
|
2021-12-25 16:10:17 +08:00
|
|
|
if err := node.chTicker.start(); err != nil {
|
2023-06-19 13:28:41 +08:00
|
|
|
log.Warn("failed to start channels time ticker", zap.String("role", typeutil.ProxyRole), zap.Error(err))
|
2021-05-29 09:55:29 +08:00
|
|
|
return err
|
|
|
|
}
|
2021-12-25 16:10:17 +08:00
|
|
|
log.Debug("start channels time ticker done", zap.String("role", typeutil.ProxyRole))
|
2021-05-29 09:55:29 +08:00
|
|
|
|
2021-05-31 17:28:31 +08:00
|
|
|
node.sendChannelsTimeTickLoop()
|
|
|
|
|
2021-01-22 09:36:18 +08:00
|
|
|
// Start callbacks
|
|
|
|
for _, cb := range node.startCallbacks {
|
|
|
|
cb()
|
|
|
|
}
|
|
|
|
|
2022-10-10 15:55:22 +08:00
|
|
|
log.Debug("update state code", zap.String("role", typeutil.ProxyRole), zap.String("State", commonpb.StateCode_Healthy.String()))
|
|
|
|
node.UpdateStateCode(commonpb.StateCode_Healthy)
|
2021-03-04 22:27:12 +08:00
|
|
|
|
2021-01-22 09:36:18 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-10-16 22:08:52 +08:00
|
|
|
// Stop stops a proxy node.
|
2021-06-22 14:40:07 +08:00
|
|
|
func (node *Proxy) Stop() error {
|
2021-01-22 09:36:18 +08:00
|
|
|
node.cancel()
|
|
|
|
|
2022-10-09 10:06:58 +08:00
|
|
|
if node.rowIDAllocator != nil {
|
|
|
|
node.rowIDAllocator.Close()
|
2021-12-13 10:01:18 +08:00
|
|
|
log.Info("close id allocator", zap.String("role", typeutil.ProxyRole))
|
2021-06-22 17:40:07 +08:00
|
|
|
}
|
2021-12-10 22:09:17 +08:00
|
|
|
|
2021-06-22 17:40:07 +08:00
|
|
|
if node.segAssigner != nil {
|
|
|
|
node.segAssigner.Close()
|
2021-12-13 10:01:18 +08:00
|
|
|
log.Info("close segment id assigner", zap.String("role", typeutil.ProxyRole))
|
2021-06-22 17:40:07 +08:00
|
|
|
}
|
2021-12-10 22:09:17 +08:00
|
|
|
|
2021-06-22 17:40:07 +08:00
|
|
|
if node.sched != nil {
|
|
|
|
node.sched.Close()
|
2021-12-13 10:01:18 +08:00
|
|
|
log.Info("close scheduler", zap.String("role", typeutil.ProxyRole))
|
2021-06-22 17:40:07 +08:00
|
|
|
}
|
2021-12-10 22:09:17 +08:00
|
|
|
|
2021-06-22 17:40:07 +08:00
|
|
|
if node.chTicker != nil {
|
|
|
|
err := node.chTicker.close()
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2021-12-13 10:01:18 +08:00
|
|
|
log.Info("close channels time ticker", zap.String("role", typeutil.ProxyRole))
|
2021-05-29 09:55:29 +08:00
|
|
|
}
|
2021-01-22 09:36:18 +08:00
|
|
|
|
|
|
|
node.wg.Wait()
|
|
|
|
|
|
|
|
for _, cb := range node.closeCallbacks {
|
|
|
|
cb()
|
|
|
|
}
|
|
|
|
|
2023-04-12 20:12:28 +08:00
|
|
|
if node.session != nil {
|
|
|
|
node.session.Stop()
|
|
|
|
}
|
2021-11-16 22:31:14 +08:00
|
|
|
|
2022-06-02 12:16:03 +08:00
|
|
|
if node.shardMgr != nil {
|
|
|
|
node.shardMgr.Close()
|
|
|
|
}
|
|
|
|
|
2022-09-27 19:18:54 +08:00
|
|
|
if node.chMgr != nil {
|
2022-10-18 19:17:27 +08:00
|
|
|
node.chMgr.removeAllDMLStream()
|
2022-09-27 19:18:54 +08:00
|
|
|
}
|
|
|
|
|
2023-06-16 18:38:39 +08:00
|
|
|
if node.lbPolicy != nil {
|
|
|
|
node.lbPolicy.Close()
|
|
|
|
}
|
|
|
|
|
2023-10-20 14:26:09 +08:00
|
|
|
if node.resourceManager != nil {
|
|
|
|
node.resourceManager.Close()
|
|
|
|
}
|
|
|
|
|
2021-11-26 11:39:16 +08:00
|
|
|
// https://github.com/milvus-io/milvus/issues/12282
|
2022-10-10 15:55:22 +08:00
|
|
|
node.UpdateStateCode(commonpb.StateCode_Abnormal)
|
2021-11-26 11:39:16 +08:00
|
|
|
|
2023-05-19 12:51:23 +08:00
|
|
|
GetConnectionManager().stop()
|
|
|
|
|
2021-01-22 09:36:18 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// AddStartCallback adds a callback in the startServer phase.
|
2021-06-22 14:40:07 +08:00
|
|
|
func (node *Proxy) AddStartCallback(callbacks ...func()) {
|
2021-01-22 09:36:18 +08:00
|
|
|
node.startCallbacks = append(node.startCallbacks, callbacks...)
|
|
|
|
}
|
|
|
|
|
2022-01-10 22:39:54 +08:00
|
|
|
// lastTick returns the last write timestamp of all pchans in this Proxy.
|
2021-06-22 14:40:07 +08:00
|
|
|
func (node *Proxy) lastTick() Timestamp {
|
2021-09-26 19:23:56 +08:00
|
|
|
return node.chTicker.getMinTick()
|
2021-01-22 09:36:18 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
// AddCloseCallback adds a callback in the Close phase.
|
2021-06-22 14:40:07 +08:00
|
|
|
func (node *Proxy) AddCloseCallback(callbacks ...func()) {
|
2021-01-22 09:36:18 +08:00
|
|
|
node.closeCallbacks = append(node.closeCallbacks, callbacks...)
|
|
|
|
}
|
2021-01-29 09:27:26 +08:00
|
|
|
|
2022-11-04 14:25:38 +08:00
|
|
|
func (node *Proxy) SetAddress(address string) {
|
|
|
|
node.address = address
|
|
|
|
}
|
|
|
|
|
2023-01-12 19:49:40 +08:00
|
|
|
func (node *Proxy) GetAddress() string {
|
|
|
|
return node.address
|
|
|
|
}
|
|
|
|
|
2021-12-29 14:35:21 +08:00
|
|
|
// SetEtcdClient sets etcd client for proxy.
|
|
|
|
func (node *Proxy) SetEtcdClient(client *clientv3.Client) {
|
|
|
|
node.etcdCli = client
|
|
|
|
}
|
|
|
|
|
2021-12-23 15:01:53 +08:00
|
|
|
// SetRootCoordClient sets RootCoord client for proxy.
|
2023-09-26 09:57:25 +08:00
|
|
|
func (node *Proxy) SetRootCoordClient(cli types.RootCoordClient) {
|
2021-06-21 17:28:03 +08:00
|
|
|
node.rootCoord = cli
|
2021-01-29 09:27:26 +08:00
|
|
|
}
|
|
|
|
|
2021-12-06 19:15:48 +08:00
|
|
|
// SetDataCoordClient sets DataCoord client for proxy.
|
2023-09-26 09:57:25 +08:00
|
|
|
func (node *Proxy) SetDataCoordClient(cli types.DataCoordClient) {
|
2021-06-21 18:22:13 +08:00
|
|
|
node.dataCoord = cli
|
2021-01-29 09:27:26 +08:00
|
|
|
}
|
|
|
|
|
2021-12-06 19:15:48 +08:00
|
|
|
// SetQueryCoordClient sets QueryCoord client for proxy.
|
2023-09-26 09:57:25 +08:00
|
|
|
func (node *Proxy) SetQueryCoordClient(cli types.QueryCoordClient) {
|
2021-06-22 16:44:09 +08:00
|
|
|
node.queryCoord = cli
|
2021-01-29 09:27:26 +08:00
|
|
|
}
|
2022-09-16 09:56:47 +08:00
|
|
|
|
2023-09-26 09:57:25 +08:00
|
|
|
func (node *Proxy) SetQueryNodeCreator(f func(ctx context.Context, addr string, nodeID int64) (types.QueryNodeClient, error)) {
|
2023-06-13 10:20:37 +08:00
|
|
|
node.shardMgr.SetClientCreatorFunc(f)
|
2023-01-12 19:49:40 +08:00
|
|
|
}
|
|
|
|
|
2022-09-16 09:56:47 +08:00
|
|
|
// GetRateLimiter returns the rateLimiter in Proxy.
|
|
|
|
func (node *Proxy) GetRateLimiter() (types.Limiter, error) {
|
|
|
|
if node.multiRateLimiter == nil {
|
|
|
|
return nil, fmt.Errorf("nil rate limiter in Proxy")
|
|
|
|
}
|
|
|
|
return node.multiRateLimiter, nil
|
|
|
|
}
|