2021-10-15 18:03:25 +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 15:16:33 +08:00
|
|
|
// with the License. You may obtain a copy of the License at
|
|
|
|
//
|
2021-10-15 18:03:25 +08:00
|
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
2021-04-19 15:16:33 +08:00
|
|
|
//
|
2021-10-15 18:03:25 +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 15:16:33 +08:00
|
|
|
|
2021-05-08 15:24:12 +08:00
|
|
|
// Package datanode implements data persistence logic.
|
|
|
|
//
|
2021-06-11 17:53:37 +08:00
|
|
|
// Data node persists insert logs into persistent storage like minIO/S3.
|
2021-01-19 11:37:16 +08:00
|
|
|
package datanode
|
|
|
|
|
|
|
|
import (
|
|
|
|
"context"
|
2021-02-03 17:30:10 +08:00
|
|
|
"fmt"
|
2021-01-19 11:37:16 +08:00
|
|
|
"io"
|
2021-03-08 15:25:55 +08:00
|
|
|
"math/rand"
|
2022-03-17 17:17:22 +08:00
|
|
|
"os"
|
2021-05-25 15:35:37 +08:00
|
|
|
"sync"
|
2021-02-04 20:31:23 +08:00
|
|
|
"sync/atomic"
|
2021-11-22 16:23:17 +08:00
|
|
|
"syscall"
|
2021-01-24 21:20:11 +08:00
|
|
|
"time"
|
2021-01-19 11:37:16 +08:00
|
|
|
|
2023-02-26 11:31:49 +08:00
|
|
|
"github.com/cockroachdb/errors"
|
2022-12-12 10:57:22 +08:00
|
|
|
clientv3 "go.etcd.io/etcd/client/v3"
|
|
|
|
"go.uber.org/zap"
|
|
|
|
|
2023-06-09 01:28:37 +08:00
|
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
2023-03-23 19:43:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/datanode/allocator"
|
2023-10-13 09:55:34 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/datanode/broker"
|
2021-11-01 10:19:55 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/kv"
|
2021-08-13 10:50:09 +08:00
|
|
|
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
2022-03-17 18:03:23 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/storage"
|
2021-12-23 18:39:11 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/types"
|
2022-04-07 22:05:32 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/dependency"
|
2021-12-23 18:39:11 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
2023-04-06 19:14:32 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/log"
|
2023-06-26 17:52:44 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/metrics"
|
2023-04-06 19:14:32 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/mq/msgdispatcher"
|
|
|
|
"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/retry"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
2021-01-19 11:37:16 +08:00
|
|
|
)
|
|
|
|
|
2021-01-24 21:20:11 +08:00
|
|
|
const (
|
2021-10-23 16:57:35 +08:00
|
|
|
// ConnectEtcdMaxRetryTime is used to limit the max retry time for connection etcd
|
2022-05-05 09:31:51 +08:00
|
|
|
ConnectEtcdMaxRetryTime = 100
|
2021-01-24 21:20:11 +08:00
|
|
|
)
|
2021-01-22 09:36:40 +08:00
|
|
|
|
2022-09-26 18:06:54 +08:00
|
|
|
var getFlowGraphServiceAttempts = uint(50)
|
|
|
|
|
2021-10-03 19:46:05 +08:00
|
|
|
// makes sure DataNode implements types.DataNode
|
|
|
|
var _ types.DataNode = (*DataNode)(nil)
|
|
|
|
|
2022-01-13 14:21:34 +08:00
|
|
|
// Params from config.yaml
|
2022-11-04 14:25:38 +08:00
|
|
|
var Params *paramtable.ComponentParam = paramtable.Get()
|
2021-12-23 18:39:11 +08:00
|
|
|
|
2021-06-29 17:34:13 +08:00
|
|
|
// DataNode communicates with outside services and unioun all
|
|
|
|
// services in datanode package.
|
2021-05-08 15:24:12 +08:00
|
|
|
//
|
2021-06-29 17:34:13 +08:00
|
|
|
// DataNode implements `types.Component`, `types.DataNode` interfaces.
|
2021-05-08 15:24:12 +08:00
|
|
|
//
|
2022-10-14 15:15:24 +08:00
|
|
|
// `etcdCli` is a connection of etcd
|
|
|
|
// `rootCoord` is a grpc client of root coordinator.
|
|
|
|
// `dataCoord` is a grpc client of data service.
|
2023-09-27 11:07:25 +08:00
|
|
|
// `stateCode` is current statement of this data node, indicating whether it's healthy.
|
2022-10-14 15:15:24 +08:00
|
|
|
//
|
|
|
|
// `clearSignal` is a signal channel for releasing the flowgraph resources.
|
|
|
|
// `segmentCache` stores all flushing and flushed segments.
|
2021-03-05 20:41:34 +08:00
|
|
|
type DataNode struct {
|
2022-10-10 15:55:22 +08:00
|
|
|
ctx context.Context
|
|
|
|
cancel context.CancelFunc
|
|
|
|
Role string
|
|
|
|
stateCode atomic.Value // commonpb.StateCode_Initializing
|
2022-01-13 14:21:34 +08:00
|
|
|
flowgraphManager *flowgraphManager
|
2023-05-18 11:43:23 +08:00
|
|
|
eventManagerMap *typeutil.ConcurrentMap[string, *channelEventManager]
|
2021-09-28 18:22:16 +08:00
|
|
|
|
2021-11-25 09:43:15 +08:00
|
|
|
clearSignal chan string // vchannel name
|
2021-11-08 19:49:07 +08:00
|
|
|
segmentCache *Cache
|
|
|
|
compactionExecutor *compactionExecutor
|
2023-06-14 14:16:38 +08:00
|
|
|
timeTickSender *timeTickSender
|
2021-01-24 21:20:11 +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
|
2023-10-13 09:55:34 +08:00
|
|
|
broker broker.Broker
|
2021-01-24 21:20:11 +08:00
|
|
|
|
2023-09-21 09:45:27 +08:00
|
|
|
// call once
|
2023-03-23 19:43:57 +08:00
|
|
|
initOnce sync.Once
|
2023-05-11 13:51:20 +08:00
|
|
|
startOnce sync.Once
|
2023-07-28 10:11:08 +08:00
|
|
|
stopOnce sync.Once
|
2023-09-19 15:27:38 +08:00
|
|
|
stopWaiter sync.WaitGroup
|
2023-03-23 19:43:57 +08:00
|
|
|
sessionMu sync.Mutex // to fix data race
|
|
|
|
session *sessionutil.Session
|
2023-06-26 09:20:44 +08:00
|
|
|
watchKv kv.WatchKV
|
2023-03-23 19:43:57 +08:00
|
|
|
chunkManager storage.ChunkManager
|
|
|
|
allocator allocator.Allocator
|
2021-05-21 19:28:52 +08:00
|
|
|
|
2021-03-05 20:41:34 +08:00
|
|
|
closer io.Closer
|
2021-02-08 14:30:54 +08:00
|
|
|
|
2023-02-13 16:38:33 +08:00
|
|
|
dispClient msgdispatcher.Client
|
|
|
|
factory dependency.Factory
|
2023-07-11 15:18:28 +08:00
|
|
|
|
|
|
|
reportImportRetryTimes uint // unitest set this value to 1 to save time, default is 10
|
2021-03-05 20:41:34 +08:00
|
|
|
}
|
2021-01-24 21:20:11 +08:00
|
|
|
|
2021-05-08 15:24:12 +08:00
|
|
|
// NewDataNode will return a DataNode with abnormal state.
|
2022-04-07 22:05:32 +08:00
|
|
|
func NewDataNode(ctx context.Context, factory dependency.Factory) *DataNode {
|
2021-03-08 15:25:55 +08:00
|
|
|
rand.Seed(time.Now().UnixNano())
|
2021-02-01 11:44:02 +08:00
|
|
|
ctx2, cancel2 := context.WithCancel(ctx)
|
2021-01-19 11:37:16 +08:00
|
|
|
node := &DataNode{
|
2021-06-29 17:34:13 +08:00
|
|
|
ctx: ctx2,
|
|
|
|
cancel: cancel2,
|
|
|
|
Role: typeutil.DataNodeRole,
|
2021-02-04 17:47:19 +08:00
|
|
|
|
2021-11-08 19:49:07 +08:00
|
|
|
rootCoord: nil,
|
|
|
|
dataCoord: nil,
|
2022-04-07 22:05:32 +08:00
|
|
|
factory: factory,
|
2021-11-08 19:49:07 +08:00
|
|
|
segmentCache: newCache(),
|
|
|
|
compactionExecutor: newCompactionExecutor(),
|
2021-05-25 15:35:37 +08:00
|
|
|
|
2023-05-18 11:43:23 +08:00
|
|
|
eventManagerMap: typeutil.NewConcurrentMap[string, *channelEventManager](),
|
2022-01-13 14:21:34 +08:00
|
|
|
flowgraphManager: newFlowgraphManager(),
|
|
|
|
clearSignal: make(chan string, 100),
|
2023-07-11 15:18:28 +08:00
|
|
|
|
|
|
|
reportImportRetryTimes: 10,
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
2022-10-10 15:55:22 +08:00
|
|
|
node.UpdateStateCode(commonpb.StateCode_Abnormal)
|
2021-01-19 11:37:16 +08:00
|
|
|
return node
|
|
|
|
}
|
|
|
|
|
2022-11-04 14:25:38 +08:00
|
|
|
func (node *DataNode) SetAddress(address string) {
|
|
|
|
node.address = address
|
|
|
|
}
|
|
|
|
|
2023-01-12 19:49:40 +08:00
|
|
|
func (node *DataNode) GetAddress() string {
|
|
|
|
return node.address
|
|
|
|
}
|
|
|
|
|
2022-01-13 14:21:34 +08:00
|
|
|
// SetEtcdClient sets etcd client for DataNode
|
2021-12-29 14:35:21 +08:00
|
|
|
func (node *DataNode) SetEtcdClient(etcdCli *clientv3.Client) {
|
|
|
|
node.etcdCli = etcdCli
|
|
|
|
}
|
|
|
|
|
2023-09-26 09:57:25 +08:00
|
|
|
// SetRootCoordClient sets RootCoord's grpc client, error is returned if repeatedly set.
|
|
|
|
func (node *DataNode) SetRootCoordClient(rc types.RootCoordClient) error {
|
2021-02-03 17:30:10 +08:00
|
|
|
switch {
|
2021-06-21 17:28:03 +08:00
|
|
|
case rc == nil, node.rootCoord != nil:
|
2022-10-09 15:54:58 +08:00
|
|
|
return errors.New("nil parameter or repeatedly set")
|
2021-02-03 17:30:10 +08:00
|
|
|
default:
|
2021-06-21 17:28:03 +08:00
|
|
|
node.rootCoord = rc
|
2021-02-03 17:30:10 +08:00
|
|
|
return nil
|
|
|
|
}
|
2021-01-26 14:46:54 +08:00
|
|
|
}
|
|
|
|
|
2023-09-26 09:57:25 +08:00
|
|
|
// SetDataCoordClient sets data service's grpc client, error is returned if repeatedly set.
|
|
|
|
func (node *DataNode) SetDataCoordClient(ds types.DataCoordClient) error {
|
2021-02-03 17:30:10 +08:00
|
|
|
switch {
|
2021-06-21 18:22:13 +08:00
|
|
|
case ds == nil, node.dataCoord != nil:
|
2022-10-09 15:54:58 +08:00
|
|
|
return errors.New("nil parameter or repeatedly set")
|
2021-02-03 17:30:10 +08:00
|
|
|
default:
|
2021-06-21 18:22:13 +08:00
|
|
|
node.dataCoord = ds
|
2021-02-03 17:30:10 +08:00
|
|
|
return nil
|
|
|
|
}
|
2021-01-26 14:46:54 +08:00
|
|
|
}
|
|
|
|
|
2021-06-29 17:34:13 +08:00
|
|
|
// Register register datanode to etcd
|
2021-05-25 15:06:05 +08:00
|
|
|
func (node *DataNode) 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.DataNodeRole).Inc()
|
|
|
|
log.Info("DataNode Register Finished")
|
2021-10-14 16:40:35 +08:00
|
|
|
// Start liveness check
|
2023-04-12 20:12:28 +08:00
|
|
|
node.session.LivenessCheck(node.ctx, func() {
|
2023-01-12 19:49:40 +08:00
|
|
|
log.Error("Data Node disconnected from etcd, process will exit", zap.Int64("Server Id", node.GetSession().ServerID))
|
2021-10-30 10:24:38 +08:00
|
|
|
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.DataNodeRole).Dec()
|
2021-11-22 16:23:17 +08:00
|
|
|
// manually send signal to starter goroutine
|
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-14 16:40:35 +08:00
|
|
|
})
|
2021-06-16 19:03:57 +08:00
|
|
|
|
2021-12-15 11:47:10 +08:00
|
|
|
return nil
|
|
|
|
}
|
2021-09-17 21:32:47 +08:00
|
|
|
|
2021-12-15 11:47:10 +08:00
|
|
|
func (node *DataNode) initSession() error {
|
2022-11-17 18:59:09 +08:00
|
|
|
node.session = sessionutil.NewSession(node.ctx, Params.EtcdCfg.MetaRootPath.GetValue(), node.etcdCli)
|
2021-12-15 11:47:10 +08:00
|
|
|
if node.session == nil {
|
|
|
|
return errors.New("failed to initialize session")
|
|
|
|
}
|
2022-11-04 14:25:38 +08:00
|
|
|
node.session.Init(typeutil.DataNodeRole, node.address, false, true)
|
2021-05-25 15:06:05 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2022-09-16 09:56:47 +08:00
|
|
|
// initRateCollector creates and starts rateCollector in QueryNode.
|
|
|
|
func (node *DataNode) initRateCollector() error {
|
2023-04-26 10:16:36 +08:00
|
|
|
err := initGlobalRateCollector()
|
2022-09-16 09:56:47 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
rateCol.Register(metricsinfo.InsertConsumeThroughput)
|
|
|
|
rateCol.Register(metricsinfo.DeleteConsumeThroughput)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-01-22 09:36:40 +08:00
|
|
|
func (node *DataNode) Init() error {
|
2023-01-12 19:49:40 +08:00
|
|
|
var initError error
|
|
|
|
node.initOnce.Do(func() {
|
|
|
|
logutil.Logger(node.ctx).Info("DataNode server initializing",
|
|
|
|
zap.String("TimeTickChannelName", Params.CommonCfg.DataCoordTimeTick.GetValue()),
|
|
|
|
)
|
|
|
|
if err := node.initSession(); err != nil {
|
|
|
|
log.Error("DataNode server init session failed", zap.Error(err))
|
|
|
|
initError = err
|
|
|
|
return
|
|
|
|
}
|
2021-12-15 11:47:10 +08:00
|
|
|
|
2023-10-13 09:55:34 +08:00
|
|
|
node.broker = broker.NewCoordBroker(node.rootCoord, node.dataCoord)
|
|
|
|
|
2023-01-12 19:49:40 +08:00
|
|
|
err := node.initRateCollector()
|
|
|
|
if err != nil {
|
|
|
|
log.Error("DataNode server init rateCollector failed", zap.Int64("node ID", paramtable.GetNodeID()), zap.Error(err))
|
|
|
|
initError = err
|
|
|
|
return
|
|
|
|
}
|
|
|
|
log.Info("DataNode server init rateCollector done", zap.Int64("node ID", paramtable.GetNodeID()))
|
2022-09-16 09:56:47 +08:00
|
|
|
|
2023-02-13 16:38:33 +08:00
|
|
|
node.dispClient = msgdispatcher.NewClient(node.factory, typeutil.DataNodeRole, paramtable.GetNodeID())
|
|
|
|
log.Info("DataNode server init dispatcher client done", zap.Int64("node ID", paramtable.GetNodeID()))
|
|
|
|
|
2023-03-23 19:43:57 +08:00
|
|
|
alloc, err := allocator.New(context.Background(), node.rootCoord, paramtable.GetNodeID())
|
2023-01-12 19:49:40 +08:00
|
|
|
if err != nil {
|
|
|
|
log.Error("failed to create id allocator",
|
|
|
|
zap.Error(err),
|
|
|
|
zap.String("role", typeutil.DataNodeRole), zap.Int64("DataNode ID", paramtable.GetNodeID()))
|
|
|
|
initError = err
|
|
|
|
return
|
|
|
|
}
|
2023-03-23 19:43:57 +08:00
|
|
|
node.allocator = alloc
|
2022-06-14 16:18:09 +08:00
|
|
|
|
2023-01-12 19:49:40 +08:00
|
|
|
node.factory.Init(Params)
|
|
|
|
log.Info("DataNode server init succeeded",
|
|
|
|
zap.String("MsgChannelSubName", Params.CommonCfg.DataNodeSubName.GetValue()))
|
|
|
|
})
|
|
|
|
return initError
|
2021-05-25 15:35:37 +08:00
|
|
|
}
|
|
|
|
|
2021-09-10 14:46:00 +08:00
|
|
|
// handleChannelEvt handles event from kv watch event
|
2021-08-13 10:50:09 +08:00
|
|
|
func (node *DataNode) handleChannelEvt(evt *clientv3.Event) {
|
2022-01-13 14:21:34 +08:00
|
|
|
var e *event
|
2021-08-13 10:50:09 +08:00
|
|
|
switch evt.Type {
|
|
|
|
case clientv3.EventTypePut: // datacoord shall put channels needs to be watched here
|
2022-01-13 14:21:34 +08:00
|
|
|
e = &event{
|
|
|
|
eventType: putEventType,
|
2022-03-09 15:39:58 +08:00
|
|
|
version: evt.Kv.Version,
|
2022-01-13 14:21:34 +08:00
|
|
|
}
|
|
|
|
|
2021-08-13 10:50:09 +08:00
|
|
|
case clientv3.EventTypeDelete:
|
2022-01-13 14:21:34 +08:00
|
|
|
e = &event{
|
|
|
|
eventType: deleteEventType,
|
2022-03-09 15:39:58 +08:00
|
|
|
version: evt.Kv.Version,
|
2022-01-13 14:21:34 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
node.handleWatchInfo(e, string(evt.Kv.Key), evt.Kv.Value)
|
|
|
|
}
|
|
|
|
|
2022-07-06 13:54:21 +08:00
|
|
|
// tryToReleaseFlowgraph tries to release a flowgraph
|
|
|
|
func (node *DataNode) tryToReleaseFlowgraph(vChanName string) {
|
2022-09-26 15:40:53 +08:00
|
|
|
log.Info("try to release flowgraph", zap.String("vChanName", vChanName))
|
2022-01-13 14:21:34 +08:00
|
|
|
node.flowgraphManager.release(vChanName)
|
2021-01-24 21:20:11 +08:00
|
|
|
}
|
|
|
|
|
2021-06-07 11:25:37 +08:00
|
|
|
// BackGroundGC runs in background to release datanode resources
|
2022-04-20 16:45:41 +08:00
|
|
|
// GOOSE TODO: remove background GC, using ToRelease for drop-collection after #15846
|
2021-11-25 09:43:15 +08:00
|
|
|
func (node *DataNode) BackGroundGC(vChannelCh <-chan string) {
|
2023-09-19 15:27:38 +08:00
|
|
|
defer node.stopWaiter.Done()
|
2021-06-07 11:25:37 +08:00
|
|
|
log.Info("DataNode Background GC Start")
|
|
|
|
for {
|
|
|
|
select {
|
2022-01-13 14:21:34 +08:00
|
|
|
case vchanName := <-vChannelCh:
|
2022-04-20 16:45:41 +08:00
|
|
|
node.tryToReleaseFlowgraph(vchanName)
|
2021-06-07 11:25:37 +08:00
|
|
|
case <-node.ctx.Done():
|
2022-02-07 16:55:44 +08:00
|
|
|
log.Warn("DataNode context done, exiting background GC")
|
2021-06-07 11:25:37 +08:00
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-05-28 14:54:31 +08:00
|
|
|
// Start will update DataNode state to HEALTHY
|
2021-01-24 21:20:11 +08:00
|
|
|
func (node *DataNode) Start() error {
|
2023-05-11 13:51:20 +08:00
|
|
|
var startErr error
|
|
|
|
node.startOnce.Do(func() {
|
|
|
|
if err := node.allocator.Start(); err != nil {
|
|
|
|
log.Error("failed to start id allocator", zap.Error(err), zap.String("role", typeutil.DataNodeRole))
|
|
|
|
startErr = err
|
|
|
|
return
|
|
|
|
}
|
|
|
|
log.Info("start id allocator done", zap.String("role", typeutil.DataNodeRole))
|
|
|
|
|
2023-10-13 09:55:34 +08:00
|
|
|
/*
|
|
|
|
rep, err := node.rootCoord.AllocTimestamp(node.ctx, &rootcoordpb.AllocTimestampRequest{
|
|
|
|
Base: commonpbutil.NewMsgBase(
|
|
|
|
commonpbutil.WithMsgType(commonpb.MsgType_RequestTSO),
|
|
|
|
commonpbutil.WithMsgID(0),
|
|
|
|
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
|
|
|
),
|
|
|
|
Count: 1,
|
|
|
|
})
|
|
|
|
if err != nil || rep.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success {
|
|
|
|
log.Warn("fail to alloc timestamp", zap.Any("rep", rep), zap.Error(err))
|
|
|
|
startErr = errors.New("DataNode fail to alloc timestamp")
|
|
|
|
return
|
|
|
|
}*/
|
2021-08-13 10:50:09 +08:00
|
|
|
|
2023-05-11 13:51:20 +08:00
|
|
|
connectEtcdFn := func() error {
|
|
|
|
etcdKV := etcdkv.NewEtcdKV(node.etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
|
|
|
|
node.watchKv = etcdKV
|
|
|
|
return nil
|
|
|
|
}
|
2023-10-13 09:55:34 +08:00
|
|
|
err := retry.Do(node.ctx, connectEtcdFn, retry.Attempts(ConnectEtcdMaxRetryTime))
|
2023-05-11 13:51:20 +08:00
|
|
|
if err != nil {
|
|
|
|
startErr = errors.New("DataNode fail to connect etcd")
|
|
|
|
return
|
|
|
|
}
|
2021-06-07 13:58:37 +08:00
|
|
|
|
2023-05-11 13:51:20 +08:00
|
|
|
chunkManager, err := node.factory.NewPersistentStorageChunkManager(node.ctx)
|
|
|
|
if err != nil {
|
|
|
|
startErr = err
|
|
|
|
return
|
|
|
|
}
|
2021-11-08 19:49:07 +08:00
|
|
|
|
2023-05-11 13:51:20 +08:00
|
|
|
node.chunkManager = chunkManager
|
2021-11-08 19:49:07 +08:00
|
|
|
|
2023-09-19 15:27:38 +08:00
|
|
|
node.stopWaiter.Add(1)
|
2023-05-11 13:51:20 +08:00
|
|
|
go node.BackGroundGC(node.clearSignal)
|
2021-09-17 21:32:47 +08:00
|
|
|
|
2023-05-11 13:51:20 +08:00
|
|
|
go node.compactionExecutor.start(node.ctx)
|
2021-11-08 19:49:07 +08:00
|
|
|
|
2023-06-14 14:16:38 +08:00
|
|
|
if Params.DataNodeCfg.DataNodeTimeTickByRPC.GetAsBool() {
|
2023-10-13 09:55:34 +08:00
|
|
|
node.timeTickSender = newTimeTickSender(node.broker, node.session.ServerID)
|
2023-06-14 14:16:38 +08:00
|
|
|
go node.timeTickSender.start(node.ctx)
|
|
|
|
}
|
|
|
|
|
2023-09-19 15:27:38 +08:00
|
|
|
node.stopWaiter.Add(1)
|
2023-05-11 13:51:20 +08:00
|
|
|
// Start node watch node
|
|
|
|
go node.StartWatchChannels(node.ctx)
|
2021-12-15 11:47:10 +08:00
|
|
|
|
2023-10-08 21:37:33 +08:00
|
|
|
node.stopWaiter.Add(1)
|
|
|
|
go node.flowgraphManager.start(&node.stopWaiter)
|
2023-03-21 21:37:56 +08:00
|
|
|
|
2023-05-11 13:51:20 +08:00
|
|
|
node.UpdateStateCode(commonpb.StateCode_Healthy)
|
|
|
|
})
|
|
|
|
return startErr
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
|
|
|
|
2021-05-28 14:54:31 +08:00
|
|
|
// UpdateStateCode updates datanode's state code
|
2022-10-10 15:55:22 +08:00
|
|
|
func (node *DataNode) UpdateStateCode(code commonpb.StateCode) {
|
|
|
|
node.stateCode.Store(code)
|
2021-02-23 11:40:30 +08:00
|
|
|
}
|
|
|
|
|
2021-10-09 10:10:59 +08:00
|
|
|
// GetStateCode return datanode's state code
|
2022-10-10 15:55:22 +08:00
|
|
|
func (node *DataNode) GetStateCode() commonpb.StateCode {
|
|
|
|
return node.stateCode.Load().(commonpb.StateCode)
|
2021-10-09 10:10:59 +08:00
|
|
|
}
|
|
|
|
|
2021-09-01 10:13:15 +08:00
|
|
|
func (node *DataNode) isHealthy() bool {
|
2022-10-10 15:55:22 +08:00
|
|
|
return node.GetStateCode() == commonpb.StateCode_Healthy
|
2021-09-01 10:13:15 +08:00
|
|
|
}
|
|
|
|
|
2022-07-21 15:52:27 +08:00
|
|
|
// ReadyToFlush tells whether DataNode is ready for flushing
|
2021-05-29 18:04:30 +08:00
|
|
|
func (node *DataNode) ReadyToFlush() error {
|
2022-07-21 15:52:27 +08:00
|
|
|
if !node.isHealthy() {
|
2021-05-29 18:04:30 +08:00
|
|
|
return errors.New("DataNode not in HEALTHY state")
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-09-24 20:44:05 +08:00
|
|
|
// Stop will release DataNode resources and shutdown datanode
|
2021-01-22 09:36:40 +08:00
|
|
|
func (node *DataNode) Stop() error {
|
2023-07-28 10:11:08 +08:00
|
|
|
node.stopOnce.Do(func() {
|
|
|
|
node.cancel()
|
|
|
|
// https://github.com/milvus-io/milvus/issues/12282
|
|
|
|
node.UpdateStateCode(commonpb.StateCode_Abnormal)
|
|
|
|
node.flowgraphManager.close()
|
2023-05-18 11:43:23 +08:00
|
|
|
|
2023-07-28 10:11:08 +08:00
|
|
|
node.eventManagerMap.Range(func(_ string, m *channelEventManager) bool {
|
|
|
|
m.Close()
|
|
|
|
return true
|
|
|
|
})
|
2022-06-14 16:18:09 +08:00
|
|
|
|
2023-07-28 10:11:08 +08:00
|
|
|
if node.allocator != nil {
|
|
|
|
log.Info("close id allocator", zap.String("role", typeutil.DataNodeRole))
|
|
|
|
node.allocator.Close()
|
2021-10-07 22:16:56 +08:00
|
|
|
}
|
2021-11-16 22:31:14 +08:00
|
|
|
|
2023-07-28 10:11:08 +08:00
|
|
|
if node.closer != nil {
|
|
|
|
node.closer.Close()
|
|
|
|
}
|
2021-11-26 11:39:16 +08:00
|
|
|
|
2023-07-28 10:11:08 +08:00
|
|
|
if node.session != nil {
|
|
|
|
node.session.Stop()
|
|
|
|
}
|
|
|
|
|
2023-09-19 15:27:38 +08:00
|
|
|
node.stopWaiter.Wait()
|
2023-07-28 10:11:08 +08:00
|
|
|
})
|
2021-01-22 09:36:40 +08:00
|
|
|
return nil
|
2021-01-26 14:46:54 +08:00
|
|
|
}
|
2023-01-12 19:49:40 +08:00
|
|
|
|
|
|
|
// to fix data race
|
|
|
|
func (node *DataNode) SetSession(session *sessionutil.Session) {
|
|
|
|
node.sessionMu.Lock()
|
|
|
|
defer node.sessionMu.Unlock()
|
|
|
|
node.session = session
|
|
|
|
}
|
|
|
|
|
|
|
|
// to fix data race
|
|
|
|
func (node *DataNode) GetSession() *sessionutil.Session {
|
|
|
|
node.sessionMu.Lock()
|
|
|
|
defer node.sessionMu.Unlock()
|
|
|
|
return node.session
|
|
|
|
}
|