mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-06 09:08:43 +08:00
enhance: Speed up CDC scheduling (#44564)
Make CDC watch etcd replicate pchannel meta instead of listing them periodically. issue: https://github.com/milvus-io/milvus/issues/44123 --------- Signed-off-by: bigsheeper <yihao.dai@zilliz.com>
This commit is contained in:
parent
fc46668812
commit
5ad8a29c0b
@ -19,6 +19,6 @@ package controller
|
||||
// Controller controls and schedules the CDC process.
|
||||
// It will periodically update the replications by the replicate configuration.
|
||||
type Controller interface {
|
||||
Start()
|
||||
Start() error
|
||||
Stop()
|
||||
}
|
||||
|
||||
@ -18,70 +18,135 @@ package controllerimpl
|
||||
|
||||
import (
|
||||
"context"
|
||||
"path"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"go.etcd.io/etcd/api/v3/mvccpb"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/cdc/resource"
|
||||
"github.com/milvus-io/milvus/internal/metastore/kv/streamingcoord"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||
)
|
||||
|
||||
const checkInterval = 10 * time.Second
|
||||
|
||||
type controller struct {
|
||||
ctx context.Context
|
||||
wg sync.WaitGroup
|
||||
stopOnce sync.Once
|
||||
stopChan chan struct{}
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
wg sync.WaitGroup
|
||||
|
||||
prefix string
|
||||
}
|
||||
|
||||
func NewController() *controller {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
return &controller{
|
||||
ctx: context.Background(),
|
||||
stopChan: make(chan struct{}),
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
prefix: path.Join(
|
||||
paramtable.Get().EtcdCfg.MetaRootPath.GetValue(),
|
||||
streamingcoord.ReplicatePChannelMetaPrefix,
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
func (c *controller) Start() {
|
||||
log.Ctx(c.ctx).Info("CDC controller started")
|
||||
func (c *controller) Start() error {
|
||||
c.startWatchLoop()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *controller) recoverReplicatePChannelMeta(replicatePChannels []*streamingpb.ReplicatePChannelMeta) {
|
||||
currentClusterID := paramtable.Get().CommonCfg.ClusterPrefix.GetValue()
|
||||
for _, replicate := range replicatePChannels {
|
||||
if !strings.Contains(replicate.GetSourceChannelName(), currentClusterID) {
|
||||
// current cluster is not source cluster, skip create replicator
|
||||
continue
|
||||
}
|
||||
log.Info("recover replicate pchannel meta",
|
||||
zap.String("sourceChannel", replicate.GetSourceChannelName()),
|
||||
zap.String("targetChannel", replicate.GetTargetChannelName()),
|
||||
)
|
||||
// Replicate manager ensures the idempotency of the creation.
|
||||
resource.Resource().ReplicateManagerClient().CreateReplicator(replicate)
|
||||
}
|
||||
resource.Resource().ReplicateManagerClient().RemoveOutdatedReplicators(replicatePChannels)
|
||||
}
|
||||
|
||||
func (c *controller) watchEvents(revision int64) clientv3.WatchChan {
|
||||
eventCh := resource.Resource().ETCD().Watch(
|
||||
c.ctx,
|
||||
c.prefix,
|
||||
clientv3.WithPrefix(),
|
||||
clientv3.WithRev(revision),
|
||||
)
|
||||
log.Ctx(c.ctx).Info("succeed to watch replicate pchannel meta events",
|
||||
zap.Int64("revision", revision), zap.String("prefix", c.prefix))
|
||||
return eventCh
|
||||
}
|
||||
|
||||
func (c *controller) startWatchLoop() {
|
||||
c.wg.Add(1)
|
||||
go func() {
|
||||
defer c.wg.Done()
|
||||
timer := time.NewTicker(checkInterval)
|
||||
defer timer.Stop()
|
||||
for {
|
||||
select {
|
||||
case <-c.stopChan:
|
||||
return
|
||||
case <-timer.C:
|
||||
c.run()
|
||||
channels, revision, err := ListReplicatePChannels(c.ctx, resource.Resource().ETCD(), c.prefix)
|
||||
if err != nil && c.ctx.Err() == nil {
|
||||
log.Ctx(c.ctx).Warn("failed to list replicate pchannels", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
c.recoverReplicatePChannelMeta(channels)
|
||||
eventCh := c.watchEvents(revision)
|
||||
err = c.watchLoop(eventCh)
|
||||
if err == nil {
|
||||
break
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
func (c *controller) watchLoop(eventCh clientv3.WatchChan) error {
|
||||
for {
|
||||
select {
|
||||
case <-c.ctx.Done():
|
||||
return nil
|
||||
case event, ok := <-eventCh:
|
||||
if !ok {
|
||||
panic("etcd event channel closed")
|
||||
}
|
||||
if err := event.Err(); err != nil {
|
||||
log.Warn("etcd event error", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
for _, e := range event.Events {
|
||||
replicate := MustParseReplicateChannelFromEvent(e)
|
||||
log.Info("handle replicate pchannel event",
|
||||
zap.String("sourceChannel", replicate.GetSourceChannelName()),
|
||||
zap.String("targetChannel", replicate.GetTargetChannelName()),
|
||||
zap.String("eventType", e.Type.String()),
|
||||
)
|
||||
switch e.Type {
|
||||
case mvccpb.PUT:
|
||||
currentClusterID := paramtable.Get().CommonCfg.ClusterPrefix.GetValue()
|
||||
if !strings.Contains(replicate.GetSourceChannelName(), currentClusterID) {
|
||||
// current cluster is not source cluster, skip create replicator
|
||||
continue
|
||||
}
|
||||
resource.Resource().ReplicateManagerClient().CreateReplicator(replicate)
|
||||
case mvccpb.DELETE:
|
||||
resource.Resource().ReplicateManagerClient().RemoveReplicator(replicate)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *controller) Stop() {
|
||||
c.stopOnce.Do(func() {
|
||||
log.Ctx(c.ctx).Info("CDC controller stopping...")
|
||||
close(c.stopChan)
|
||||
c.wg.Wait()
|
||||
resource.Resource().ReplicateManagerClient().Close()
|
||||
log.Ctx(c.ctx).Info("CDC controller stopped")
|
||||
})
|
||||
}
|
||||
|
||||
func (c *controller) run() {
|
||||
targetReplicatePChannels, err := resource.Resource().ReplicationCatalog().ListReplicatePChannels(c.ctx)
|
||||
if err != nil {
|
||||
log.Ctx(c.ctx).Error("failed to get replicate pchannels", zap.Error(err))
|
||||
return
|
||||
}
|
||||
// create replicators for all replicate pchannels
|
||||
for _, replicatePChannel := range targetReplicatePChannels {
|
||||
resource.Resource().ReplicateManagerClient().CreateReplicator(replicatePChannel)
|
||||
}
|
||||
|
||||
// remove out of target replicators
|
||||
resource.Resource().ReplicateManagerClient().RemoveOutOfTargetReplicators(targetReplicatePChannels)
|
||||
log.Ctx(c.ctx).Info("stop CDC controller...")
|
||||
c.cancel()
|
||||
c.wg.Wait()
|
||||
resource.Resource().ReplicateManagerClient().Close()
|
||||
log.Ctx(c.ctx).Info("CDC controller stopped")
|
||||
}
|
||||
|
||||
@ -19,69 +19,71 @@ package controllerimpl
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"go.etcd.io/etcd/api/v3/mvccpb"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"google.golang.org/protobuf/proto"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/cdc/replication"
|
||||
"github.com/milvus-io/milvus/internal/cdc/resource"
|
||||
"github.com/milvus-io/milvus/internal/mocks/mock_metastore"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
)
|
||||
|
||||
func TestController_StartAndStop(t *testing.T) {
|
||||
mockReplicateManagerClient := replication.NewMockReplicateManagerClient(t)
|
||||
mockReplicateManagerClient.EXPECT().Close().Return()
|
||||
resource.InitForTest(t,
|
||||
resource.OptReplicateManagerClient(mockReplicateManagerClient),
|
||||
)
|
||||
|
||||
ctrl := NewController()
|
||||
assert.NotPanics(t, func() {
|
||||
ctrl.Start()
|
||||
})
|
||||
assert.NotPanics(t, func() {
|
||||
ctrl.Stop()
|
||||
})
|
||||
}
|
||||
|
||||
func TestController_Run(t *testing.T) {
|
||||
func TestController_StartAndStop_WithEvents(t *testing.T) {
|
||||
mockReplicateManagerClient := replication.NewMockReplicateManagerClient(t)
|
||||
mockReplicateManagerClient.EXPECT().Close().Return()
|
||||
|
||||
replicatePChannels := []*streamingpb.ReplicatePChannelMeta{
|
||||
{
|
||||
SourceChannelName: "test-source-channel-1",
|
||||
TargetChannelName: "test-target-channel-1",
|
||||
// Create test data
|
||||
replicateMeta := &streamingpb.ReplicatePChannelMeta{
|
||||
SourceChannelName: "by-dev-test-source-channel",
|
||||
TargetChannelName: "by-dev-test-target-channel",
|
||||
}
|
||||
metaBytes, _ := proto.Marshal(replicateMeta)
|
||||
|
||||
// Create mock events
|
||||
putEvent := &clientv3.Event{
|
||||
Type: mvccpb.PUT,
|
||||
Kv: &mvccpb.KeyValue{
|
||||
Value: metaBytes,
|
||||
},
|
||||
}
|
||||
mockReplicationCatalog := mock_metastore.NewMockReplicationCatalog(t)
|
||||
mockReplicationCatalog.EXPECT().ListReplicatePChannels(mock.Anything).Return(replicatePChannels, nil)
|
||||
mockReplicateManagerClient.EXPECT().CreateReplicator(replicatePChannels[0]).Return()
|
||||
mockReplicateManagerClient.EXPECT().RemoveOutOfTargetReplicators(replicatePChannels).Return()
|
||||
|
||||
deleteEvent := &clientv3.Event{
|
||||
Type: mvccpb.DELETE,
|
||||
Kv: &mvccpb.KeyValue{
|
||||
Value: metaBytes,
|
||||
},
|
||||
}
|
||||
|
||||
eventCh := make(chan clientv3.WatchResponse, 2)
|
||||
// Send events
|
||||
go func() {
|
||||
eventCh <- clientv3.WatchResponse{
|
||||
Events: []*clientv3.Event{putEvent},
|
||||
}
|
||||
eventCh <- clientv3.WatchResponse{
|
||||
Events: []*clientv3.Event{deleteEvent},
|
||||
}
|
||||
}()
|
||||
|
||||
notifyCh := make(chan struct{}, 2)
|
||||
mockReplicateManagerClient.EXPECT().CreateReplicator(mock.Anything).RunAndReturn(func(replicate *streamingpb.ReplicatePChannelMeta) {
|
||||
notifyCh <- struct{}{}
|
||||
})
|
||||
mockReplicateManagerClient.EXPECT().RemoveReplicator(mock.Anything).RunAndReturn(func(replicate *streamingpb.ReplicatePChannelMeta) {
|
||||
notifyCh <- struct{}{}
|
||||
})
|
||||
|
||||
resource.InitForTest(t,
|
||||
resource.OptReplicateManagerClient(mockReplicateManagerClient),
|
||||
resource.OptReplicationCatalog(mockReplicationCatalog),
|
||||
)
|
||||
|
||||
ctrl := NewController()
|
||||
ctrl.Start()
|
||||
defer ctrl.Stop()
|
||||
ctrl.run()
|
||||
}
|
||||
|
||||
func TestController_RunError(t *testing.T) {
|
||||
mockReplicateManagerClient := replication.NewMockReplicateManagerClient(t)
|
||||
mockReplicateManagerClient.EXPECT().Close().Return()
|
||||
|
||||
mockReplicationCatalog := mock_metastore.NewMockReplicationCatalog(t)
|
||||
mockReplicationCatalog.EXPECT().ListReplicatePChannels(mock.Anything).Return(nil, assert.AnError)
|
||||
resource.InitForTest(t,
|
||||
resource.OptReplicateManagerClient(mockReplicateManagerClient),
|
||||
resource.OptReplicationCatalog(mockReplicationCatalog),
|
||||
)
|
||||
|
||||
ctrl := NewController()
|
||||
ctrl.Start()
|
||||
defer ctrl.Stop()
|
||||
ctrl.run()
|
||||
go ctrl.watchLoop(eventCh)
|
||||
|
||||
// Wait for events to be processed
|
||||
<-notifyCh
|
||||
<-notifyCh
|
||||
|
||||
ctrl.Stop()
|
||||
}
|
||||
|
||||
@ -0,0 +1,56 @@
|
||||
package controllerimpl
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"google.golang.org/protobuf/proto"
|
||||
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||
)
|
||||
|
||||
func ListReplicatePChannels(ctx context.Context, etcdCli *clientv3.Client, prefix string) ([]*streamingpb.ReplicatePChannelMeta, int64, error) {
|
||||
resp, err := get(ctx, etcdCli, prefix)
|
||||
if err != nil {
|
||||
return nil, 0, err
|
||||
}
|
||||
keys := make([]string, 0, resp.Count)
|
||||
values := make([]string, 0, resp.Count)
|
||||
for _, kv := range resp.Kvs {
|
||||
keys = append(keys, string(kv.Key))
|
||||
values = append(values, string(kv.Value))
|
||||
}
|
||||
channels := make([]*streamingpb.ReplicatePChannelMeta, 0, len(values))
|
||||
for k, value := range values {
|
||||
info := &streamingpb.ReplicatePChannelMeta{}
|
||||
err = proto.Unmarshal([]byte(value), info)
|
||||
if err != nil {
|
||||
return nil, 0, errors.Wrapf(err, "unmarshal replicate pchannel meta %s failed", keys[k])
|
||||
}
|
||||
channels = append(channels, info)
|
||||
}
|
||||
return channels, resp.Header.Revision, nil
|
||||
}
|
||||
|
||||
func MustParseReplicateChannelFromEvent(e *clientv3.Event) *streamingpb.ReplicatePChannelMeta {
|
||||
meta := &streamingpb.ReplicatePChannelMeta{}
|
||||
err := proto.Unmarshal(e.Kv.Value, meta)
|
||||
if err != nil {
|
||||
panic(fmt.Sprintf("failed to unmarshal replicate pchannel meta: %v", err))
|
||||
}
|
||||
return meta
|
||||
}
|
||||
|
||||
func get(ctx context.Context, etcdCli *clientv3.Client, prefix string) (*clientv3.GetResponse, error) {
|
||||
ctx1, cancel := context.WithTimeout(ctx, 30*time.Second)
|
||||
defer cancel()
|
||||
|
||||
opts := []clientv3.OpOption{
|
||||
clientv3.WithPrefix(),
|
||||
clientv3.WithSort(clientv3.SortByKey, clientv3.SortAscend),
|
||||
}
|
||||
return etcdCli.Get(ctx1, prefix, opts...)
|
||||
}
|
||||
@ -18,8 +18,21 @@ func (_m *MockController) EXPECT() *MockController_Expecter {
|
||||
}
|
||||
|
||||
// Start provides a mock function with no fields
|
||||
func (_m *MockController) Start() {
|
||||
_m.Called()
|
||||
func (_m *MockController) Start() error {
|
||||
ret := _m.Called()
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for Start")
|
||||
}
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func() error); ok {
|
||||
r0 = rf()
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockController_Start_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Start'
|
||||
@ -39,13 +52,13 @@ func (_c *MockController_Start_Call) Run(run func()) *MockController_Start_Call
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockController_Start_Call) Return() *MockController_Start_Call {
|
||||
_c.Call.Return()
|
||||
func (_c *MockController_Start_Call) Return(_a0 error) *MockController_Start_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockController_Start_Call) RunAndReturn(run func()) *MockController_Start_Call {
|
||||
_c.Run(run)
|
||||
func (_c *MockController_Start_Call) RunAndReturn(run func() error) *MockController_Start_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
|
||||
@ -85,35 +85,68 @@ func (_c *MockReplicateManagerClient_CreateReplicator_Call) RunAndReturn(run fun
|
||||
return _c
|
||||
}
|
||||
|
||||
// RemoveOutOfTargetReplicators provides a mock function with given fields: targetReplicatePChannels
|
||||
func (_m *MockReplicateManagerClient) RemoveOutOfTargetReplicators(targetReplicatePChannels []*streamingpb.ReplicatePChannelMeta) {
|
||||
_m.Called(targetReplicatePChannels)
|
||||
// RemoveOutdatedReplicators provides a mock function with given fields: aliveReplicates
|
||||
func (_m *MockReplicateManagerClient) RemoveOutdatedReplicators(aliveReplicates []*streamingpb.ReplicatePChannelMeta) {
|
||||
_m.Called(aliveReplicates)
|
||||
}
|
||||
|
||||
// MockReplicateManagerClient_RemoveOutOfTargetReplicators_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RemoveOutOfTargetReplicators'
|
||||
type MockReplicateManagerClient_RemoveOutOfTargetReplicators_Call struct {
|
||||
// MockReplicateManagerClient_RemoveOutdatedReplicators_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RemoveOutdatedReplicators'
|
||||
type MockReplicateManagerClient_RemoveOutdatedReplicators_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// RemoveOutOfTargetReplicators is a helper method to define mock.On call
|
||||
// - targetReplicatePChannels []*streamingpb.ReplicatePChannelMeta
|
||||
func (_e *MockReplicateManagerClient_Expecter) RemoveOutOfTargetReplicators(targetReplicatePChannels interface{}) *MockReplicateManagerClient_RemoveOutOfTargetReplicators_Call {
|
||||
return &MockReplicateManagerClient_RemoveOutOfTargetReplicators_Call{Call: _e.mock.On("RemoveOutOfTargetReplicators", targetReplicatePChannels)}
|
||||
// RemoveOutdatedReplicators is a helper method to define mock.On call
|
||||
// - aliveReplicates []*streamingpb.ReplicatePChannelMeta
|
||||
func (_e *MockReplicateManagerClient_Expecter) RemoveOutdatedReplicators(aliveReplicates interface{}) *MockReplicateManagerClient_RemoveOutdatedReplicators_Call {
|
||||
return &MockReplicateManagerClient_RemoveOutdatedReplicators_Call{Call: _e.mock.On("RemoveOutdatedReplicators", aliveReplicates)}
|
||||
}
|
||||
|
||||
func (_c *MockReplicateManagerClient_RemoveOutOfTargetReplicators_Call) Run(run func(targetReplicatePChannels []*streamingpb.ReplicatePChannelMeta)) *MockReplicateManagerClient_RemoveOutOfTargetReplicators_Call {
|
||||
func (_c *MockReplicateManagerClient_RemoveOutdatedReplicators_Call) Run(run func(aliveReplicates []*streamingpb.ReplicatePChannelMeta)) *MockReplicateManagerClient_RemoveOutdatedReplicators_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].([]*streamingpb.ReplicatePChannelMeta))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockReplicateManagerClient_RemoveOutOfTargetReplicators_Call) Return() *MockReplicateManagerClient_RemoveOutOfTargetReplicators_Call {
|
||||
func (_c *MockReplicateManagerClient_RemoveOutdatedReplicators_Call) Return() *MockReplicateManagerClient_RemoveOutdatedReplicators_Call {
|
||||
_c.Call.Return()
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockReplicateManagerClient_RemoveOutOfTargetReplicators_Call) RunAndReturn(run func([]*streamingpb.ReplicatePChannelMeta)) *MockReplicateManagerClient_RemoveOutOfTargetReplicators_Call {
|
||||
func (_c *MockReplicateManagerClient_RemoveOutdatedReplicators_Call) RunAndReturn(run func([]*streamingpb.ReplicatePChannelMeta)) *MockReplicateManagerClient_RemoveOutdatedReplicators_Call {
|
||||
_c.Run(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// RemoveReplicator provides a mock function with given fields: replicateInfo
|
||||
func (_m *MockReplicateManagerClient) RemoveReplicator(replicateInfo *streamingpb.ReplicatePChannelMeta) {
|
||||
_m.Called(replicateInfo)
|
||||
}
|
||||
|
||||
// MockReplicateManagerClient_RemoveReplicator_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RemoveReplicator'
|
||||
type MockReplicateManagerClient_RemoveReplicator_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// RemoveReplicator is a helper method to define mock.On call
|
||||
// - replicateInfo *streamingpb.ReplicatePChannelMeta
|
||||
func (_e *MockReplicateManagerClient_Expecter) RemoveReplicator(replicateInfo interface{}) *MockReplicateManagerClient_RemoveReplicator_Call {
|
||||
return &MockReplicateManagerClient_RemoveReplicator_Call{Call: _e.mock.On("RemoveReplicator", replicateInfo)}
|
||||
}
|
||||
|
||||
func (_c *MockReplicateManagerClient_RemoveReplicator_Call) Run(run func(replicateInfo *streamingpb.ReplicatePChannelMeta)) *MockReplicateManagerClient_RemoveReplicator_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(*streamingpb.ReplicatePChannelMeta))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockReplicateManagerClient_RemoveReplicator_Call) Return() *MockReplicateManagerClient_RemoveReplicator_Call {
|
||||
_c.Call.Return()
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockReplicateManagerClient_RemoveReplicator_Call) RunAndReturn(run func(*streamingpb.ReplicatePChannelMeta)) *MockReplicateManagerClient_RemoveReplicator_Call {
|
||||
_c.Run(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
@ -23,8 +23,11 @@ type ReplicateManagerClient interface {
|
||||
// CreateReplicator creates a new replicator for the replicate pchannel.
|
||||
CreateReplicator(replicateInfo *streamingpb.ReplicatePChannelMeta)
|
||||
|
||||
// RemoveOutOfTargetReplicators removes replicators that are not in the target replicate pchannels.
|
||||
RemoveOutOfTargetReplicators(targetReplicatePChannels []*streamingpb.ReplicatePChannelMeta)
|
||||
// RemoveReplicator removes a replicator for the replicate pchannel.
|
||||
RemoveReplicator(replicateInfo *streamingpb.ReplicatePChannelMeta)
|
||||
|
||||
// RemoveOutdatedReplicators removes the outdated replicators.
|
||||
RemoveOutdatedReplicators(aliveReplicates []*streamingpb.ReplicatePChannelMeta)
|
||||
|
||||
// Close closes the replicate manager client.
|
||||
Close()
|
||||
|
||||
@ -20,7 +20,6 @@ import (
|
||||
"context"
|
||||
"strings"
|
||||
|
||||
"github.com/samber/lo"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/metastore/kv/streamingcoord"
|
||||
@ -53,7 +52,7 @@ func (r *replicateManager) CreateReplicator(replicateInfo *streamingpb.Replicate
|
||||
)
|
||||
currentClusterID := paramtable.Get().CommonCfg.ClusterPrefix.GetValue()
|
||||
if !strings.Contains(replicateInfo.GetSourceChannelName(), currentClusterID) {
|
||||
// current cluster is not source cluster, skip create replicator
|
||||
// should be checked by controller, here is a redundant check
|
||||
return
|
||||
}
|
||||
replicatorKey := streamingcoord.BuildReplicatePChannelMetaKey(replicateInfo)
|
||||
@ -69,17 +68,37 @@ func (r *replicateManager) CreateReplicator(replicateInfo *streamingpb.Replicate
|
||||
logger.Info("created replicator for replicate pchannel")
|
||||
}
|
||||
|
||||
func (r *replicateManager) RemoveOutOfTargetReplicators(targetReplicatePChannels []*streamingpb.ReplicatePChannelMeta) {
|
||||
targets := lo.KeyBy(targetReplicatePChannels, streamingcoord.BuildReplicatePChannelMetaKey)
|
||||
func (r *replicateManager) RemoveReplicator(replicateInfo *streamingpb.ReplicatePChannelMeta) {
|
||||
logger := log.With(
|
||||
zap.String("sourceChannel", replicateInfo.GetSourceChannelName()),
|
||||
zap.String("targetChannel", replicateInfo.GetTargetChannelName()),
|
||||
)
|
||||
replicatorKey := streamingcoord.BuildReplicatePChannelMetaKey(replicateInfo)
|
||||
replicator, ok := r.replicators[replicatorKey]
|
||||
if !ok {
|
||||
logger.Info("replicator not found, skip remove")
|
||||
return
|
||||
}
|
||||
replicator.StopReplicate()
|
||||
delete(r.replicators, replicatorKey)
|
||||
delete(r.replicatorPChannels, replicatorKey)
|
||||
logger.Info("removed replicator for replicate pchannel")
|
||||
}
|
||||
|
||||
func (r *replicateManager) RemoveOutdatedReplicators(aliveReplicates []*streamingpb.ReplicatePChannelMeta) {
|
||||
alivesMap := make(map[string]struct{})
|
||||
for _, replicate := range aliveReplicates {
|
||||
alivesMap[streamingcoord.BuildReplicatePChannelMetaKey(replicate)] = struct{}{}
|
||||
}
|
||||
for replicatorKey, replicator := range r.replicators {
|
||||
if pchannelMeta, ok := targets[replicatorKey]; !ok {
|
||||
if _, ok := alivesMap[replicatorKey]; !ok {
|
||||
replicator.StopReplicate()
|
||||
info := r.replicatorPChannels[replicatorKey]
|
||||
delete(r.replicators, replicatorKey)
|
||||
delete(r.replicatorPChannels, replicatorKey)
|
||||
log.Info("removed replicator due to out of target",
|
||||
zap.String("sourceChannel", pchannelMeta.GetSourceChannelName()),
|
||||
zap.String("targetChannel", pchannelMeta.GetTargetChannelName()),
|
||||
)
|
||||
log.Info("removed replicator for replicate pchannel",
|
||||
zap.String("sourceChannel", info.GetSourceChannelName()),
|
||||
zap.String("targetChannel", info.GetTargetChannelName()))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -19,6 +19,8 @@ package resource
|
||||
import (
|
||||
"reflect"
|
||||
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/cdc/cluster"
|
||||
"github.com/milvus-io/milvus/internal/cdc/controller"
|
||||
"github.com/milvus-io/milvus/internal/cdc/replication"
|
||||
@ -39,6 +41,13 @@ func OptMetaKV(metaKV kv.MetaKv) optResourceInit {
|
||||
}
|
||||
}
|
||||
|
||||
// OptETCD provides the etcd client to the resource.
|
||||
func OptETCD(etcd *clientv3.Client) optResourceInit {
|
||||
return func(r *resourceImpl) {
|
||||
r.etcdClient = etcd
|
||||
}
|
||||
}
|
||||
|
||||
// OptReplicateManagerClient provides the replicate manager client to the resource.
|
||||
func OptReplicateManagerClient(replicateManagerClient replication.ReplicateManagerClient) optResourceInit {
|
||||
return func(r *resourceImpl) {
|
||||
@ -78,6 +87,7 @@ func Init(opts ...optResourceInit) {
|
||||
newR.clusterClient = cluster.NewClusterClient()
|
||||
|
||||
assertNotNil(newR.MetaKV())
|
||||
assertNotNil(newR.ETCD())
|
||||
assertNotNil(newR.ReplicationCatalog())
|
||||
assertNotNil(newR.ClusterClient())
|
||||
assertNotNil(newR.ReplicateManagerClient())
|
||||
@ -97,6 +107,7 @@ func Resource() *resourceImpl {
|
||||
// All utility on it is concurrent-safe and singleton.
|
||||
type resourceImpl struct {
|
||||
metaKV kv.MetaKv
|
||||
etcdClient *clientv3.Client
|
||||
catalog metastore.ReplicationCatalog
|
||||
clusterClient cluster.ClusterClient
|
||||
replicateManagerClient replication.ReplicateManagerClient
|
||||
@ -108,6 +119,11 @@ func (r *resourceImpl) MetaKV() kv.MetaKv {
|
||||
return r.metaKV
|
||||
}
|
||||
|
||||
// ETCD returns the etcd client.
|
||||
func (r *resourceImpl) ETCD() *clientv3.Client {
|
||||
return r.etcdClient
|
||||
}
|
||||
|
||||
// ReplicationCatalog returns the replication catalog.
|
||||
func (r *resourceImpl) ReplicationCatalog() metastore.ReplicationCatalog {
|
||||
return r.catalog
|
||||
|
||||
@ -6,6 +6,8 @@ package resource
|
||||
import (
|
||||
"testing"
|
||||
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/cdc/cluster"
|
||||
"github.com/milvus-io/milvus/internal/cdc/controller"
|
||||
"github.com/milvus-io/milvus/internal/cdc/replication"
|
||||
@ -22,6 +24,9 @@ func InitForTest(t *testing.T, opts ...optResourceInit) {
|
||||
if r.metaKV == nil {
|
||||
r.metaKV = mock_kv.NewMockMetaKv(t)
|
||||
}
|
||||
if r.etcdClient == nil {
|
||||
r.etcdClient = &clientv3.Client{}
|
||||
}
|
||||
if r.catalog == nil {
|
||||
r.catalog = mock_metastore.NewMockReplicationCatalog(t)
|
||||
}
|
||||
|
||||
@ -19,6 +19,8 @@ package cdc
|
||||
import (
|
||||
"context"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/cdc/resource"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
)
|
||||
@ -42,7 +44,11 @@ func (svr *CDCServer) Init() error {
|
||||
|
||||
// Start starts CDCServer.
|
||||
func (svr *CDCServer) Start() error {
|
||||
resource.Resource().Controller().Start()
|
||||
err := resource.Resource().Controller().Start()
|
||||
if err != nil {
|
||||
log.Ctx(svr.ctx).Error("start CDC controller failed", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
log.Ctx(svr.ctx).Info("CDCServer start successfully")
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -32,14 +32,12 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/cdc/replication/replicatemanager"
|
||||
"github.com/milvus-io/milvus/internal/cdc/resource"
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
tikvkv "github.com/milvus-io/milvus/internal/kv/tikv"
|
||||
"github.com/milvus-io/milvus/internal/util/componentutil"
|
||||
kvfactory "github.com/milvus-io/milvus/internal/util/dependency/kv"
|
||||
"github.com/milvus-io/milvus/pkg/v2/kv"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/tikv"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
|
||||
@ -48,7 +46,7 @@ type Server struct {
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
|
||||
metaKV kv.MetaKv
|
||||
watchKV kv.WatchKV
|
||||
cdcServer *cdc.CDCServer
|
||||
|
||||
etcdCli *clientv3.Client
|
||||
@ -146,7 +144,8 @@ func (s *Server) init() (err error) {
|
||||
// Create CDC service.
|
||||
s.cdcServer = cdc.NewCDCServer(s.ctx)
|
||||
resource.Init(
|
||||
resource.OptMetaKV(s.metaKV),
|
||||
resource.OptMetaKV(s.watchKV),
|
||||
resource.OptETCD(s.etcdCli),
|
||||
resource.OptReplicateManagerClient(replicatemanager.NewReplicateManager()),
|
||||
resource.OptController(controllerimpl.NewController()),
|
||||
)
|
||||
@ -176,18 +175,11 @@ func (s *Server) initMeta() error {
|
||||
log.Info("cdc connecting to metadata store", zap.String("metaType", metaType))
|
||||
metaRootPath := ""
|
||||
if metaType == util.MetaStoreTypeTiKV {
|
||||
var err error
|
||||
s.tikvCli, err = tikv.GetTiKVClient(¶mtable.Get().TiKVCfg)
|
||||
if err != nil {
|
||||
log.Warn("cdc init tikv client failed", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
metaRootPath = params.TiKVCfg.MetaRootPath.GetValue()
|
||||
s.metaKV = tikvkv.NewTiKV(s.tikvCli, metaRootPath,
|
||||
tikvkv.WithRequestTimeout(paramtable.Get().ServiceParam.TiKVCfg.RequestTimeout.GetAsDuration(time.Millisecond)))
|
||||
// TODO: sheep, support watch operation for tikv
|
||||
panic("tikv is not supported for cdc")
|
||||
} else if metaType == util.MetaStoreTypeEtcd {
|
||||
metaRootPath = params.EtcdCfg.MetaRootPath.GetValue()
|
||||
s.metaKV = etcdkv.NewEtcdKV(s.etcdCli, metaRootPath,
|
||||
s.watchKV = etcdkv.NewEtcdKV(s.etcdCli, metaRootPath,
|
||||
etcdkv.WithRequestTimeout(paramtable.Get().ServiceParam.EtcdCfg.RequestTimeout.GetAsDuration(time.Millisecond)))
|
||||
}
|
||||
return nil
|
||||
|
||||
@ -8,6 +8,7 @@ packages:
|
||||
github.com/milvus-io/milvus/pkg/v2/kv:
|
||||
interfaces:
|
||||
MetaKv:
|
||||
WatchKV:
|
||||
github.com/milvus-io/milvus/pkg/v2/streaming/util/message:
|
||||
interfaces:
|
||||
MessageID:
|
||||
|
||||
1034
pkg/mocks/mock_kv/mock_WatchKV.go
Normal file
1034
pkg/mocks/mock_kv/mock_WatchKV.go
Normal file
File diff suppressed because it is too large
Load Diff
Loading…
x
Reference in New Issue
Block a user