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-11-01 10:19:55 +08:00
"path"
2021-08-13 10:50:09 +08:00
"strings"
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"
2021-08-13 10:50:09 +08:00
"github.com/golang/protobuf/proto"
2022-12-12 10:57:22 +08:00
v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap"
2023-03-23 19:43:57 +08:00
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/datanode/allocator"
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"
2021-04-22 14:45:57 +08:00
"github.com/milvus-io/milvus/internal/log"
2023-02-13 16:38:33 +08:00
"github.com/milvus-io/milvus/internal/mq/msgdispatcher"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
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-10-21 15:57:28 +08:00
"github.com/milvus-io/milvus/internal/util/commonpbutil"
2022-04-07 22:05:32 +08:00
"github.com/milvus-io/milvus/internal/util/dependency"
2022-02-09 18:55:46 +08:00
"github.com/milvus-io/milvus/internal/util/logutil"
2021-12-23 18:39:11 +08:00
"github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/retry"
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/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
// RPCConnectionTimeout is used to set the timeout for rpc request
2021-01-24 21:20:11 +08:00
RPCConnectionTimeout = 30 * time . Second
2021-06-21 18:08:15 +08:00
2021-10-23 16:57:35 +08:00
// MetricRequestsTotal is used to count the num of total requests
2021-06-21 18:08:15 +08:00
MetricRequestsTotal = "total"
2021-10-23 16:57:35 +08:00
// MetricRequestsSuccess is used to count the num of successful requests
2021-06-21 18:08:15 +08:00
MetricRequestsSuccess = "success"
2021-08-13 10:50:09 +08:00
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
2022-10-14 15:15:24 +08:00
2022-11-21 10:11:11 +08:00
// ImportCallTimeout is the timeout used in Import() method calls
// This value is equal to RootCoord's task expire time
ImportCallTimeout = 15 * 60 * time . Second
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
2022-09-16 09:56:47 +08:00
// rateCol is global rateCollector in DataNode.
var rateCol * rateCollector
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.
// `NodeID` is unique to each datanode.
// `State` is current statement of this data node, indicating whether it's healthy.
//
// `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
eventManagerMap sync . Map // vchannel name -> 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
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
2021-06-21 18:22:13 +08:00
rootCoord types . RootCoord
dataCoord types . DataCoord
2021-01-24 21:20:11 +08:00
2023-01-12 19:49:40 +08:00
//call once
2023-03-23 19:43:57 +08:00
initOnce sync . Once
sessionMu sync . Mutex // to fix data race
session * sessionutil . Session
watchKv kv . MetaKv
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
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
2022-01-13 14:21:34 +08:00
flowgraphManager : newFlowgraphManager ( ) ,
clearSignal : make ( chan string , 100 ) ,
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
}
2021-10-09 11:28:57 +08:00
// SetRootCoord sets RootCoord's grpc client, error is returned if repeatedly set.
2021-10-09 10:10:59 +08:00
func ( node * DataNode ) SetRootCoord ( rc types . RootCoord ) 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
}
2021-10-09 11:28:57 +08:00
// SetDataCoord sets data service's grpc client, error is returned if repeatedly set.
2021-10-09 10:10:59 +08:00
func ( node * DataNode ) SetDataCoord ( ds types . DataCoord ) 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 ( )
2021-10-14 16:40:35 +08:00
// Start liveness check
go 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 ) )
}
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 {
var err error
rateCol , err = newRateCollector ( )
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-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 ( ) ) )
2021-01-24 21:20:11 +08:00
2023-01-12 19:49:40 +08:00
} )
return initError
2021-05-25 15:35:37 +08:00
}
2021-08-13 10:50:09 +08:00
// StartWatchChannels start loop to watch channel allocation status via kv(etcd for now)
func ( node * DataNode ) StartWatchChannels ( ctx context . Context ) {
defer logutil . LogPanic ( )
// REF MEP#7 watch path should be [prefix]/channel/{node_id}/{channel_name}
2022-06-06 16:26:06 +08:00
// TODO, this is risky, we'd better watch etcd with revision rather simply a path
2023-01-12 19:49:40 +08:00
watchPrefix := path . Join ( Params . CommonCfg . DataCoordWatchSubPath . GetValue ( ) , fmt . Sprintf ( "%d" , node . GetSession ( ) . ServerID ) )
log . Info ( "Start watch channel" , zap . String ( "prefix" , watchPrefix ) )
2021-11-01 10:19:55 +08:00
evtChan := node . watchKv . WatchWithPrefix ( watchPrefix )
2021-10-13 17:02:33 +08:00
// after watch, first check all exists nodes first
2021-10-15 17:02:37 +08:00
err := node . checkWatchedList ( )
if err != nil {
log . Warn ( "StartWatchChannels failed" , zap . Error ( err ) )
return
}
2021-08-13 10:50:09 +08:00
for {
select {
case <- ctx . Done ( ) :
2022-03-02 15:35:55 +08:00
log . Info ( "watch etcd loop quit" )
2021-08-13 10:50:09 +08:00
return
2022-06-06 16:26:06 +08:00
case event , ok := <- evtChan :
if ! ok {
log . Warn ( "datanode failed to watch channel, return" )
return
}
if err := event . Err ( ) ; err != nil {
log . Warn ( "datanode watch channel canceled" , zap . Error ( event . Err ( ) ) )
2021-10-13 17:02:33 +08:00
// https://github.com/etcd-io/etcd/issues/8980
if event . Err ( ) == v3rpc . ErrCompacted {
go node . StartWatchChannels ( ctx )
return
}
2021-08-13 10:50:09 +08:00
// if watch loop return due to event canceled, the datanode is not functional anymore
2022-06-06 16:26:06 +08:00
log . Panic ( "datanode is not functional for event canceled" , zap . Error ( err ) )
2021-08-13 10:50:09 +08:00
return
}
for _ , evt := range event . Events {
2022-06-08 17:24:07 +08:00
// We need to stay in order until events enqueued
node . handleChannelEvt ( evt )
2021-08-13 10:50:09 +08:00
}
}
}
}
2021-10-13 17:02:33 +08:00
// checkWatchedList list all nodes under [prefix]/channel/{node_id} and make sure all nodeds are watched
// serves the corner case for etcd connection lost and missing some events
func ( node * DataNode ) checkWatchedList ( ) error {
// REF MEP#7 watch path should be [prefix]/channel/{node_id}/{channel_name}
2022-12-07 18:01:19 +08:00
prefix := path . Join ( Params . CommonCfg . DataCoordWatchSubPath . GetValue ( ) , fmt . Sprintf ( "%d" , paramtable . GetNodeID ( ) ) )
2021-11-01 10:19:55 +08:00
keys , values , err := node . watchKv . LoadWithPrefix ( prefix )
2021-10-13 17:02:33 +08:00
if err != nil {
return err
}
for i , val := range values {
2022-01-13 14:21:34 +08:00
node . handleWatchInfo ( & event { eventType : putEventType } , keys [ i ] , [ ] byte ( val ) )
2021-10-13 17:02:33 +08:00
}
return nil
}
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 )
}
func ( node * DataNode ) handleWatchInfo ( e * event , key string , data [ ] byte ) {
switch e . eventType {
2022-03-28 22:33:27 +08:00
case putEventType :
2022-01-13 14:21:34 +08:00
watchInfo , err := parsePutEventData ( data )
if err != nil {
log . Warn ( "fail to handle watchInfo" , zap . Int ( "event type" , e . eventType ) , zap . String ( "key" , key ) , zap . Error ( err ) )
return
}
2022-03-09 15:39:58 +08:00
if isEndWatchState ( watchInfo . State ) {
2022-10-25 19:31:30 +08:00
log . Info ( "DataNode received a PUT event with an end State" , zap . String ( "state" , watchInfo . State . String ( ) ) )
2022-03-09 15:39:58 +08:00
return
}
2023-02-15 16:20:34 +08:00
if watchInfo . Progress != 0 {
log . Info ( "DataNode received a PUT event with tickler update progress" , zap . String ( "channel" , watchInfo . Vchan . ChannelName ) , zap . Int64 ( "version" , e . version ) )
return
}
2022-01-13 14:21:34 +08:00
e . info = watchInfo
e . vChanName = watchInfo . GetVchan ( ) . GetChannelName ( )
2022-03-29 00:15:28 +08:00
log . Info ( "DataNode is handling watchInfo PUT event" , zap . String ( "key" , key ) , zap . Any ( "watch state" , watchInfo . GetState ( ) . String ( ) ) )
2022-01-13 14:21:34 +08:00
case deleteEventType :
e . vChanName = parseDeleteEventKey ( key )
2022-03-29 00:15:28 +08:00
log . Info ( "DataNode is handling watchInfo DELETE event" , zap . String ( "key" , key ) )
2022-01-13 14:21:34 +08:00
}
2022-03-09 15:39:58 +08:00
actualManager , loaded := node . eventManagerMap . LoadOrStore ( e . vChanName , newChannelEventManager (
node . handlePutEvent , node . handleDeleteEvent , retryWatchInterval ,
) )
2022-01-13 14:21:34 +08:00
if ! loaded {
actualManager . ( * channelEventManager ) . Run ( )
}
actualManager . ( * channelEventManager ) . handleEvent ( * e )
2022-04-20 16:45:41 +08:00
// Whenever a delete event comes, this eventManager will be removed from map
2022-01-13 14:21:34 +08:00
if e . eventType == deleteEventType {
if m , loaded := node . eventManagerMap . LoadAndDelete ( e . vChanName ) ; loaded {
m . ( * channelEventManager ) . Close ( )
}
2021-08-13 10:50:09 +08:00
}
}
2022-01-13 14:21:34 +08:00
func parsePutEventData ( data [ ] byte ) ( * datapb . ChannelWatchInfo , error ) {
2021-10-13 17:02:33 +08:00
watchInfo := datapb . ChannelWatchInfo { }
err := proto . Unmarshal ( data , & watchInfo )
if err != nil {
2022-01-13 14:21:34 +08:00
return nil , fmt . Errorf ( "invalid event data: fail to parse ChannelWatchInfo, err: %v" , err )
2021-10-13 17:02:33 +08:00
}
2022-01-13 14:21:34 +08:00
2021-10-13 17:02:33 +08:00
if watchInfo . Vchan == nil {
2022-01-13 14:21:34 +08:00
return nil , fmt . Errorf ( "invalid event: ChannelWatchInfo with nil VChannelInfo" )
2021-10-13 17:02:33 +08:00
}
2022-07-07 21:58:22 +08:00
reviseVChannelInfo ( watchInfo . GetVchan ( ) )
2022-01-13 14:21:34 +08:00
return & watchInfo , nil
}
2022-01-04 15:47:18 +08:00
2022-01-13 14:21:34 +08:00
func parseDeleteEventKey ( key string ) string {
parts := strings . Split ( key , "/" )
vChanName := parts [ len ( parts ) - 1 ]
return vChanName
2021-10-13 17:02:33 +08:00
}
2022-03-09 15:39:58 +08:00
func ( node * DataNode ) handlePutEvent ( watchInfo * datapb . ChannelWatchInfo , version int64 ) ( err error ) {
2022-01-13 14:21:34 +08:00
vChanName := watchInfo . GetVchan ( ) . GetChannelName ( )
2023-02-15 16:20:34 +08:00
key := path . Join ( Params . CommonCfg . DataCoordWatchSubPath . GetValue ( ) , fmt . Sprintf ( "%d" , node . GetSession ( ) . ServerID ) , vChanName )
tickler := newTickler ( version , key , watchInfo , node . watchKv , Params . DataNodeCfg . WatchEventTicklerInterval . GetAsDuration ( time . Second ) )
2022-03-09 15:39:58 +08:00
switch watchInfo . State {
case datapb . ChannelWatchState_Uncomplete , datapb . ChannelWatchState_ToWatch :
2023-02-15 16:20:34 +08:00
if err := node . flowgraphManager . addAndStart ( node , watchInfo . GetVchan ( ) , watchInfo . GetSchema ( ) , tickler ) ; err != nil {
watchInfo . State = datapb . ChannelWatchState_WatchFailure
2022-03-09 15:39:58 +08:00
return fmt . Errorf ( "fail to add and start flowgraph for vChanName: %s, err: %v" , vChanName , err )
}
2022-09-26 15:40:53 +08:00
log . Info ( "handle put event: new data sync service success" , zap . String ( "vChanName" , vChanName ) )
2022-03-09 15:39:58 +08:00
watchInfo . State = datapb . ChannelWatchState_WatchSuccess
case datapb . ChannelWatchState_ToRelease :
2022-07-06 13:54:21 +08:00
// there is no reason why we release fail
node . tryToReleaseFlowgraph ( vChanName )
watchInfo . State = datapb . ChannelWatchState_ReleaseSuccess
2021-05-08 15:24:12 +08:00
}
2022-01-13 14:21:34 +08:00
v , err := proto . Marshal ( watchInfo )
2021-10-14 10:24:33 +08:00
if err != nil {
2022-03-09 15:39:58 +08:00
return fmt . Errorf ( "fail to marshal watchInfo with state, vChanName: %s, state: %s ,err: %w" , vChanName , watchInfo . State . String ( ) , err )
2021-10-14 10:24:33 +08:00
}
2021-05-29 18:04:30 +08:00
2023-02-15 16:20:34 +08:00
success , err := node . watchKv . CompareVersionAndSwap ( key , tickler . version , string ( v ) )
// etcd error
2021-06-19 15:18:06 +08:00
if err != nil {
2022-07-06 13:54:21 +08:00
// flow graph will leak if not release, causing new datanode failed to subscribe
node . tryToReleaseFlowgraph ( vChanName )
log . Warn ( "fail to update watch state to etcd" , zap . String ( "vChanName" , vChanName ) ,
zap . String ( "state" , watchInfo . State . String ( ) ) , zap . Error ( err ) )
return err
2021-06-19 15:18:06 +08:00
}
2022-07-06 13:54:21 +08:00
// etcd valid but the states updated.
if ! success {
log . Info ( "handle put event: failed to compare version and swap, release flowgraph" ,
2022-09-26 15:40:53 +08:00
zap . String ( "key" , key ) , zap . String ( "state" , watchInfo . State . String ( ) ) ,
zap . String ( "vChanName" , vChanName ) )
2022-07-06 13:54:21 +08:00
// flow graph will leak if not release, causing new datanode failed to subscribe
node . tryToReleaseFlowgraph ( vChanName )
return nil
}
2022-09-26 15:40:53 +08:00
log . Info ( "handle put event success" , zap . String ( "key" , key ) ,
zap . String ( "state" , watchInfo . State . String ( ) ) , zap . String ( "vChanName" , vChanName ) )
2022-01-13 14:21:34 +08:00
return nil
}
2021-06-29 17:34:13 +08:00
2022-07-06 13:54:21 +08:00
func ( node * DataNode ) handleDeleteEvent ( vChanName string ) {
node . tryToReleaseFlowgraph ( vChanName )
2022-01-13 14:21:34 +08:00
}
2021-05-25 15:35:37 +08:00
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 ) {
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-03-23 19:43:57 +08:00
if err := node . allocator . Start ( ) ; err != nil {
2022-06-14 16:18:09 +08:00
log . Error ( "failed to start id allocator" , zap . Error ( err ) , zap . String ( "role" , typeutil . DataNodeRole ) )
return err
}
2022-10-25 19:31:30 +08:00
log . Info ( "start id allocator done" , zap . String ( "role" , typeutil . DataNodeRole ) )
2021-06-07 13:58:37 +08:00
2021-06-22 16:14:09 +08:00
rep , err := node . rootCoord . AllocTimestamp ( node . ctx , & rootcoordpb . AllocTimestampRequest {
2022-10-21 15:57:28 +08:00
Base : commonpbutil . NewMsgBase (
commonpbutil . WithMsgType ( commonpb . MsgType_RequestTSO ) ,
commonpbutil . WithMsgID ( 0 ) ,
2022-11-04 14:25:38 +08:00
commonpbutil . WithSourceID ( paramtable . GetNodeID ( ) ) ,
2022-10-21 15:57:28 +08:00
) ,
2021-06-07 13:58:37 +08:00
Count : 1 ,
} )
2022-07-21 15:52:27 +08:00
if err != nil || rep . Status . ErrorCode != commonpb . ErrorCode_Success {
log . Warn ( "fail to alloc timestamp" , zap . Any ( "rep" , rep ) , zap . Error ( err ) )
return errors . New ( "DataNode fail to alloc timestamp" )
2021-08-13 10:50:09 +08:00
}
connectEtcdFn := func ( ) error {
2022-11-17 18:59:09 +08:00
etcdKV := etcdkv . NewEtcdKV ( node . etcdCli , Params . EtcdCfg . MetaRootPath . GetValue ( ) )
2021-11-01 10:19:55 +08:00
node . watchKv = etcdKV
2021-08-13 10:50:09 +08:00
return nil
}
err = retry . Do ( node . ctx , connectEtcdFn , retry . Attempts ( ConnectEtcdMaxRetryTime ) )
if err != nil {
return errors . New ( "DataNode fail to connect etcd" )
}
2021-06-07 13:58:37 +08:00
2022-09-23 14:40:51 +08:00
chunkManager , err := node . factory . NewPersistentStorageChunkManager ( node . ctx )
2021-11-08 19:49:07 +08:00
if err != nil {
return err
}
2022-03-17 18:03:23 +08:00
node . chunkManager = chunkManager
2021-11-08 19:49:07 +08:00
2021-06-07 11:25:37 +08:00
go node . BackGroundGC ( node . clearSignal )
2021-09-17 21:32:47 +08:00
2021-11-08 19:49:07 +08:00
go node . compactionExecutor . start ( node . ctx )
2021-12-15 11:47:10 +08:00
// Start node watch node
go node . StartWatchChannels ( node . ctx )
2023-03-21 21:37:56 +08:00
go node . flowgraphManager . start ( )
2022-10-10 15:55:22 +08:00
node . UpdateStateCode ( commonpb . StateCode_Healthy )
2021-02-04 17:47:19 +08:00
return nil
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 {
2021-12-08 14:19:04 +08:00
// https://github.com/milvus-io/milvus/issues/12282
2022-10-10 15:55:22 +08:00
node . UpdateStateCode ( commonpb . StateCode_Abnormal )
2022-01-13 14:21:34 +08:00
node . flowgraphManager . dropAll ( )
2023-03-21 21:37:56 +08:00
node . flowgraphManager . stop ( )
2021-01-19 11:37:16 +08:00
2023-03-23 19:43:57 +08:00
node . cancel ( )
if node . allocator != nil {
2022-06-14 16:18:09 +08:00
log . Info ( "close id allocator" , zap . String ( "role" , typeutil . DataNodeRole ) )
2023-03-23 19:43:57 +08:00
node . allocator . Close ( )
2022-06-14 16:18:09 +08:00
}
2021-01-19 11:37:16 +08:00
if node . closer != nil {
2021-10-07 22:16:56 +08:00
err := node . closer . Close ( )
if err != nil {
return err
}
2021-01-19 11:37:16 +08:00
}
2021-11-16 22:31:14 +08:00
node . session . Revoke ( time . Second )
2021-11-26 11:39:16 +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
}