2022-01-13 14:21:34 +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
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// 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.
package datanode
import (
2023-09-19 15:27:38 +08:00
"context"
"fmt"
"path"
"strings"
2022-03-09 15:39:58 +08:00
"sync"
2022-01-13 14:21:34 +08:00
"time"
2023-02-15 16:20:34 +08:00
"github.com/golang/protobuf/proto"
2023-09-19 15:27:38 +08:00
v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes"
2023-02-15 16:20:34 +08:00
"go.uber.org/atomic"
2022-01-13 14:21:34 +08:00
"go.uber.org/zap"
2023-04-06 19:14:32 +08:00
"github.com/milvus-io/milvus/internal/kv"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/pkg/log"
2023-09-19 15:27:38 +08:00
"github.com/milvus-io/milvus/pkg/util/logutil"
"github.com/milvus-io/milvus/pkg/util/paramtable"
2022-01-13 14:21:34 +08:00
)
const retryWatchInterval = 20 * time . Second
2023-09-19 15:27:38 +08:00
// StartWatchChannels start loop to watch channel allocation status via kv(etcd for now)
func ( node * DataNode ) StartWatchChannels ( ctx context . Context ) {
defer node . stopWaiter . Done ( )
defer logutil . LogPanic ( )
// REF MEP#7 watch path should be [prefix]/channel/{node_id}/{channel_name}
// TODO, this is risky, we'd better watch etcd with revision rather simply a path
watchPrefix := path . Join ( Params . CommonCfg . DataCoordWatchSubPath . GetValue ( ) , fmt . Sprintf ( "%d" , node . GetSession ( ) . ServerID ) )
log . Info ( "Start watch channel" , zap . String ( "prefix" , watchPrefix ) )
evtChan := node . watchKv . WatchWithPrefix ( watchPrefix )
// after watch, first check all exists nodes first
err := node . checkWatchedList ( )
if err != nil {
log . Warn ( "StartWatchChannels failed" , zap . Error ( err ) )
return
}
for {
select {
case <- ctx . Done ( ) :
log . Info ( "watch etcd loop quit" )
return
case event , ok := <- evtChan :
if ! ok {
log . Warn ( "datanode failed to watch channel, return" )
go node . StartWatchChannels ( ctx )
return
}
if err := event . Err ( ) ; err != nil {
log . Warn ( "datanode watch channel canceled" , zap . Error ( event . Err ( ) ) )
// https://github.com/etcd-io/etcd/issues/8980
if event . Err ( ) == v3rpc . ErrCompacted {
go node . StartWatchChannels ( ctx )
return
}
// if watch loop return due to event canceled, the datanode is not functional anymore
log . Panic ( "datanode is not functional for event canceled" , zap . Error ( err ) )
return
}
for _ , evt := range event . Events {
// We need to stay in order until events enqueued
node . handleChannelEvt ( evt )
}
}
}
}
2023-11-13 10:50:17 +08:00
// checkWatchedList list all nodes under [prefix]/channel/{node_id} and make sure all nodes are watched
2023-09-19 15:27:38 +08:00
// 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}
prefix := path . Join ( Params . CommonCfg . DataCoordWatchSubPath . GetValue ( ) , fmt . Sprintf ( "%d" , paramtable . GetNodeID ( ) ) )
keys , values , err := node . watchKv . LoadWithPrefix ( prefix )
if err != nil {
return err
}
for i , val := range values {
node . handleWatchInfo ( & event { eventType : putEventType } , keys [ i ] , [ ] byte ( val ) )
}
return nil
}
func ( node * DataNode ) handleWatchInfo ( e * event , key string , data [ ] byte ) {
switch e . eventType {
case putEventType :
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
}
if isEndWatchState ( watchInfo . State ) {
log . Info ( "DataNode received a PUT event with an end State" , zap . String ( "state" , watchInfo . State . String ( ) ) )
return
}
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
}
e . info = watchInfo
e . vChanName = watchInfo . GetVchan ( ) . GetChannelName ( )
log . Info ( "DataNode is handling watchInfo PUT event" , zap . String ( "key" , key ) , zap . Any ( "watch state" , watchInfo . GetState ( ) . String ( ) ) )
case deleteEventType :
e . vChanName = parseDeleteEventKey ( key )
log . Info ( "DataNode is handling watchInfo DELETE event" , zap . String ( "key" , key ) )
}
actualManager , loaded := node . eventManagerMap . GetOrInsert ( e . vChanName , newChannelEventManager (
node . handlePutEvent , node . handleDeleteEvent , retryWatchInterval ,
) )
if ! loaded {
actualManager . Run ( )
}
actualManager . handleEvent ( * e )
// Whenever a delete event comes, this eventManager will be removed from map
if e . eventType == deleteEventType {
if m , loaded := node . eventManagerMap . GetAndRemove ( e . vChanName ) ; loaded {
m . Close ( )
}
}
}
func parsePutEventData ( data [ ] byte ) ( * datapb . ChannelWatchInfo , error ) {
watchInfo := datapb . ChannelWatchInfo { }
err := proto . Unmarshal ( data , & watchInfo )
if err != nil {
return nil , fmt . Errorf ( "invalid event data: fail to parse ChannelWatchInfo, err: %v" , err )
}
if watchInfo . Vchan == nil {
return nil , fmt . Errorf ( "invalid event: ChannelWatchInfo with nil VChannelInfo" )
}
reviseVChannelInfo ( watchInfo . GetVchan ( ) )
return & watchInfo , nil
}
func parseDeleteEventKey ( key string ) string {
parts := strings . Split ( key , "/" )
vChanName := parts [ len ( parts ) - 1 ]
return vChanName
}
func ( node * DataNode ) handlePutEvent ( watchInfo * datapb . ChannelWatchInfo , version int64 ) ( err error ) {
vChanName := watchInfo . GetVchan ( ) . GetChannelName ( )
key := path . Join ( Params . CommonCfg . DataCoordWatchSubPath . GetValue ( ) , fmt . Sprintf ( "%d" , node . GetSession ( ) . ServerID ) , vChanName )
2023-09-27 11:07:25 +08:00
tickler := newEtcdTickler ( version , key , watchInfo , node . watchKv , Params . DataNodeCfg . WatchEventTicklerInterval . GetAsDuration ( time . Second ) )
2023-09-19 15:27:38 +08:00
switch watchInfo . State {
case datapb . ChannelWatchState_Uncomplete , datapb . ChannelWatchState_ToWatch :
2023-09-27 11:07:25 +08:00
if err := node . flowgraphManager . addAndStartWithEtcdTickler ( node , watchInfo . GetVchan ( ) , watchInfo . GetSchema ( ) , tickler ) ; err != nil {
2023-09-19 15:27:38 +08:00
log . Warn ( "handle put event: new data sync service failed" , zap . String ( "vChanName" , vChanName ) , zap . Error ( err ) )
watchInfo . State = datapb . ChannelWatchState_WatchFailure
} else {
log . Info ( "handle put event: new data sync service success" , zap . String ( "vChanName" , vChanName ) )
watchInfo . State = datapb . ChannelWatchState_WatchSuccess
}
case datapb . ChannelWatchState_ToRelease :
// there is no reason why we release fail
node . tryToReleaseFlowgraph ( vChanName )
watchInfo . State = datapb . ChannelWatchState_ReleaseSuccess
}
v , err := proto . Marshal ( watchInfo )
if err != nil {
return fmt . Errorf ( "fail to marshal watchInfo with state, vChanName: %s, state: %s ,err: %w" , vChanName , watchInfo . State . String ( ) , err )
}
success , err := node . watchKv . CompareVersionAndSwap ( key , tickler . version , string ( v ) )
// etcd error
if err != nil {
// 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
}
// etcd valid but the states updated.
if ! success {
log . Info ( "handle put event: failed to compare version and swap, release flowgraph" ,
zap . String ( "key" , key ) , zap . String ( "state" , watchInfo . State . String ( ) ) ,
zap . String ( "vChanName" , vChanName ) )
// flow graph will leak if not release, causing new datanode failed to subscribe
node . tryToReleaseFlowgraph ( vChanName )
return nil
}
log . Info ( "handle put event success" , zap . String ( "key" , key ) ,
zap . String ( "state" , watchInfo . State . String ( ) ) , zap . String ( "vChanName" , vChanName ) )
return nil
}
func ( node * DataNode ) handleDeleteEvent ( vChanName string ) {
node . tryToReleaseFlowgraph ( vChanName )
}
2022-01-13 14:21:34 +08:00
type event struct {
eventType int
vChanName string
2022-03-09 15:39:58 +08:00
version int64
2022-01-13 14:21:34 +08:00
info * datapb . ChannelWatchInfo
}
type channelEventManager struct {
2022-03-09 15:39:58 +08:00
sync . Once
2023-05-18 11:43:23 +08:00
wg sync . WaitGroup
2022-01-13 14:21:34 +08:00
eventChan chan event
closeChan chan struct { }
2022-03-09 15:39:58 +08:00
handlePutEvent func ( watchInfo * datapb . ChannelWatchInfo , version int64 ) error // node.handlePutEvent
2022-07-06 13:54:21 +08:00
handleDeleteEvent func ( vChanName string ) // node.handleDeleteEvent
2022-03-09 15:39:58 +08:00
retryInterval time . Duration
2022-01-13 14:21:34 +08:00
}
const (
putEventType = 1
deleteEventType = 2
)
2022-03-09 15:39:58 +08:00
func newChannelEventManager ( handlePut func ( * datapb . ChannelWatchInfo , int64 ) error ,
2023-09-21 09:45:27 +08:00
handleDel func ( string ) , retryInterval time . Duration ,
) * channelEventManager {
2022-03-09 15:39:58 +08:00
return & channelEventManager {
eventChan : make ( chan event , 10 ) ,
closeChan : make ( chan struct { } ) ,
handlePutEvent : handlePut ,
handleDeleteEvent : handleDel ,
retryInterval : retryInterval ,
}
}
2022-01-13 14:21:34 +08:00
func ( e * channelEventManager ) Run ( ) {
2023-05-18 11:43:23 +08:00
e . wg . Add ( 1 )
2022-01-13 14:21:34 +08:00
go func ( ) {
2023-05-18 11:43:23 +08:00
defer e . wg . Done ( )
2022-01-13 14:21:34 +08:00
for {
select {
case event := <- e . eventChan :
switch event . eventType {
case putEventType :
2023-07-12 17:26:30 +08:00
err := e . handlePutEvent ( event . info , event . version )
if err != nil {
// logging the error is convenient for follow-up investigation of problems
log . Warn ( "handle put event failed" , zap . String ( "vChanName" , event . vChanName ) , zap . Error ( err ) )
}
2022-01-13 14:21:34 +08:00
case deleteEventType :
e . handleDeleteEvent ( event . vChanName )
}
case <- e . closeChan :
return
}
}
} ( )
}
2023-02-15 16:20:34 +08:00
func ( e * channelEventManager ) handleEvent ( event event ) {
e . eventChan <- event
}
func ( e * channelEventManager ) Close ( ) {
e . Do ( func ( ) {
close ( e . closeChan )
2023-05-18 11:43:23 +08:00
e . wg . Wait ( )
2023-02-15 16:20:34 +08:00
} )
}
func isEndWatchState ( state datapb . ChannelWatchState ) bool {
return state != datapb . ChannelWatchState_ToWatch && // start watch
state != datapb . ChannelWatchState_ToRelease && // start release
state != datapb . ChannelWatchState_Uncomplete // legacy state, equal to ToWatch
}
2023-09-27 11:07:25 +08:00
type etcdTickler struct {
2023-02-15 16:20:34 +08:00
progress * atomic . Int32
version int64
2023-09-07 07:25:14 +08:00
kv kv . WatchKV
2023-02-15 16:20:34 +08:00
path string
watchInfo * datapb . ChannelWatchInfo
2023-08-20 21:20:24 +08:00
interval time . Duration
closeCh chan struct { }
closeWg sync . WaitGroup
2023-08-23 13:22:27 +08:00
isWatchFailed * atomic . Bool
2023-02-15 16:20:34 +08:00
}
2023-09-27 11:07:25 +08:00
func ( t * etcdTickler ) inc ( ) {
2023-02-15 16:20:34 +08:00
t . progress . Inc ( )
}
2023-09-27 11:07:25 +08:00
func ( t * etcdTickler ) watch ( ) {
2023-02-15 16:20:34 +08:00
if t . interval == 0 {
log . Info ( "zero interval, close ticler watch" ,
2023-07-14 15:56:31 +08:00
zap . String ( "channelName" , t . watchInfo . GetVchan ( ) . GetChannelName ( ) ) ,
2023-02-15 16:20:34 +08:00
)
2022-03-09 15:39:58 +08:00
return
}
2023-02-15 16:20:34 +08:00
t . closeWg . Add ( 1 )
go func ( ) {
2023-08-20 21:20:24 +08:00
defer t . closeWg . Done ( )
2023-02-23 18:59:45 +08:00
ticker := time . NewTicker ( t . interval )
defer ticker . Stop ( )
2023-02-15 16:20:34 +08:00
for {
2022-03-09 15:39:58 +08:00
select {
case <- ticker . C :
2023-02-15 16:20:34 +08:00
nowProgress := t . progress . Load ( )
if t . watchInfo . Progress == nowProgress {
continue
}
t . watchInfo . Progress = nowProgress
v , err := proto . Marshal ( t . watchInfo )
if err != nil {
log . Error ( "fail to marshal watchInfo with progress at tickler" ,
zap . String ( "vChanName" , t . watchInfo . Vchan . ChannelName ) ,
zap . Int32 ( "progree" , nowProgress ) ,
zap . Error ( err ) )
2023-08-23 13:22:27 +08:00
t . isWatchFailed . Store ( true )
2022-03-09 15:39:58 +08:00
return
}
2023-02-15 16:20:34 +08:00
success , err := t . kv . CompareVersionAndSwap ( t . path , t . version , string ( v ) )
if err != nil {
log . Error ( "tickler update failed" , zap . Error ( err ) )
continue
2022-03-09 15:39:58 +08:00
}
2023-02-15 16:20:34 +08:00
if ! success {
log . Error ( "tickler update failed: failed to compare version and swap" ,
zap . String ( "key" , t . path ) , zap . Int32 ( "progress" , nowProgress ) , zap . Int64 ( "version" , t . version ) ,
zap . String ( "vChanName" , t . watchInfo . GetVchan ( ) . ChannelName ) )
2023-08-23 13:22:27 +08:00
t . isWatchFailed . Store ( true )
2022-03-09 15:39:58 +08:00
return
}
2023-02-15 16:20:34 +08:00
log . Debug ( "tickler update success" , zap . Int32 ( "progress" , nowProgress ) , zap . Int64 ( "version" , t . version ) ,
zap . String ( "vChanName" , t . watchInfo . GetVchan ( ) . ChannelName ) )
t . version ++
case <- t . closeCh :
2022-03-09 15:39:58 +08:00
return
}
}
2023-02-15 16:20:34 +08:00
} ( )
2022-01-13 14:21:34 +08:00
}
2023-09-27 11:07:25 +08:00
func ( t * etcdTickler ) stop ( ) {
2023-02-15 16:20:34 +08:00
close ( t . closeCh )
t . closeWg . Wait ( )
2022-03-09 15:39:58 +08:00
}
2023-09-27 11:07:25 +08:00
func newEtcdTickler ( version int64 , path string , watchInfo * datapb . ChannelWatchInfo , kv kv . WatchKV , interval time . Duration ) * etcdTickler {
return & etcdTickler {
2023-08-23 13:22:27 +08:00
progress : atomic . NewInt32 ( 0 ) ,
path : path ,
kv : kv ,
watchInfo : watchInfo ,
version : version ,
interval : interval ,
closeCh : make ( chan struct { } ) ,
isWatchFailed : atomic . NewBool ( false ) ,
2023-02-15 16:20:34 +08:00
}
2022-01-13 14:21:34 +08:00
}