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:
yihao.dai 2025-10-15 10:15:59 +08:00 committed by GitHub
parent fc46668812
commit 5ad8a29c0b
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
14 changed files with 1378 additions and 133 deletions

View File

@ -19,6 +19,6 @@ package controller
// Controller controls and schedules the CDC process. // Controller controls and schedules the CDC process.
// It will periodically update the replications by the replicate configuration. // It will periodically update the replications by the replicate configuration.
type Controller interface { type Controller interface {
Start() Start() error
Stop() Stop()
} }

View File

@ -18,70 +18,135 @@ package controllerimpl
import ( import (
"context" "context"
"path"
"strings"
"sync" "sync"
"time"
"go.etcd.io/etcd/api/v3/mvccpb"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap" "go.uber.org/zap"
"github.com/milvus-io/milvus/internal/cdc/resource" "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/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 { type controller struct {
ctx context.Context ctx context.Context
cancel context.CancelFunc
wg sync.WaitGroup wg sync.WaitGroup
stopOnce sync.Once
stopChan chan struct{} prefix string
} }
func NewController() *controller { func NewController() *controller {
ctx, cancel := context.WithCancel(context.Background())
return &controller{ return &controller{
ctx: context.Background(), ctx: ctx,
stopChan: make(chan struct{}), cancel: cancel,
prefix: path.Join(
paramtable.Get().EtcdCfg.MetaRootPath.GetValue(),
streamingcoord.ReplicatePChannelMetaPrefix,
),
} }
} }
func (c *controller) Start() { func (c *controller) Start() error {
log.Ctx(c.ctx).Info("CDC controller started") 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) c.wg.Add(1)
go func() { go func() {
defer c.wg.Done() defer c.wg.Done()
timer := time.NewTicker(checkInterval)
defer timer.Stop()
for { for {
select { channels, revision, err := ListReplicatePChannels(c.ctx, resource.Resource().ETCD(), c.prefix)
case <-c.stopChan: if err != nil && c.ctx.Err() == nil {
return log.Ctx(c.ctx).Warn("failed to list replicate pchannels", zap.Error(err))
case <-timer.C: continue
c.run() }
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() { func (c *controller) Stop() {
c.stopOnce.Do(func() { log.Ctx(c.ctx).Info("stop CDC controller...")
log.Ctx(c.ctx).Info("CDC controller stopping...") c.cancel()
close(c.stopChan)
c.wg.Wait() c.wg.Wait()
resource.Resource().ReplicateManagerClient().Close() resource.Resource().ReplicateManagerClient().Close()
log.Ctx(c.ctx).Info("CDC controller stopped") 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)
} }

View File

@ -19,69 +19,71 @@ package controllerimpl
import ( import (
"testing" "testing"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock" "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/replication"
"github.com/milvus-io/milvus/internal/cdc/resource" "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" "github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
) )
func TestController_StartAndStop(t *testing.T) { func TestController_StartAndStop_WithEvents(t *testing.T) {
mockReplicateManagerClient := replication.NewMockReplicateManagerClient(t) mockReplicateManagerClient := replication.NewMockReplicateManagerClient(t)
mockReplicateManagerClient.EXPECT().Close().Return() mockReplicateManagerClient.EXPECT().Close().Return()
resource.InitForTest(t,
resource.OptReplicateManagerClient(mockReplicateManagerClient),
)
ctrl := NewController() // Create test data
assert.NotPanics(t, func() { replicateMeta := &streamingpb.ReplicatePChannelMeta{
ctrl.Start() SourceChannelName: "by-dev-test-source-channel",
}) TargetChannelName: "by-dev-test-target-channel",
assert.NotPanics(t, func() {
ctrl.Stop()
})
} }
metaBytes, _ := proto.Marshal(replicateMeta)
func TestController_Run(t *testing.T) { // Create mock events
mockReplicateManagerClient := replication.NewMockReplicateManagerClient(t) putEvent := &clientv3.Event{
mockReplicateManagerClient.EXPECT().Close().Return() Type: mvccpb.PUT,
Kv: &mvccpb.KeyValue{
replicatePChannels := []*streamingpb.ReplicatePChannelMeta{ Value: metaBytes,
{
SourceChannelName: "test-source-channel-1",
TargetChannelName: "test-target-channel-1",
}, },
} }
mockReplicationCatalog := mock_metastore.NewMockReplicationCatalog(t)
mockReplicationCatalog.EXPECT().ListReplicatePChannels(mock.Anything).Return(replicatePChannels, nil) deleteEvent := &clientv3.Event{
mockReplicateManagerClient.EXPECT().CreateReplicator(replicatePChannels[0]).Return() Type: mvccpb.DELETE,
mockReplicateManagerClient.EXPECT().RemoveOutOfTargetReplicators(replicatePChannels).Return() 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.InitForTest(t,
resource.OptReplicateManagerClient(mockReplicateManagerClient), resource.OptReplicateManagerClient(mockReplicateManagerClient),
resource.OptReplicationCatalog(mockReplicationCatalog),
) )
ctrl := NewController() ctrl := NewController()
ctrl.Start() go ctrl.watchLoop(eventCh)
defer ctrl.Stop()
ctrl.run() // Wait for events to be processed
} <-notifyCh
<-notifyCh
func TestController_RunError(t *testing.T) {
mockReplicateManagerClient := replication.NewMockReplicateManagerClient(t) ctrl.Stop()
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()
} }

View File

@ -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...)
}

View File

@ -18,8 +18,21 @@ func (_m *MockController) EXPECT() *MockController_Expecter {
} }
// Start provides a mock function with no fields // Start provides a mock function with no fields
func (_m *MockController) Start() { func (_m *MockController) Start() error {
_m.Called() 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' // 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 return _c
} }
func (_c *MockController_Start_Call) Return() *MockController_Start_Call { func (_c *MockController_Start_Call) Return(_a0 error) *MockController_Start_Call {
_c.Call.Return() _c.Call.Return(_a0)
return _c return _c
} }
func (_c *MockController_Start_Call) RunAndReturn(run func()) *MockController_Start_Call { func (_c *MockController_Start_Call) RunAndReturn(run func() error) *MockController_Start_Call {
_c.Run(run) _c.Call.Return(run)
return _c return _c
} }

View File

@ -85,35 +85,68 @@ func (_c *MockReplicateManagerClient_CreateReplicator_Call) RunAndReturn(run fun
return _c return _c
} }
// RemoveOutOfTargetReplicators provides a mock function with given fields: targetReplicatePChannels // RemoveOutdatedReplicators provides a mock function with given fields: aliveReplicates
func (_m *MockReplicateManagerClient) RemoveOutOfTargetReplicators(targetReplicatePChannels []*streamingpb.ReplicatePChannelMeta) { func (_m *MockReplicateManagerClient) RemoveOutdatedReplicators(aliveReplicates []*streamingpb.ReplicatePChannelMeta) {
_m.Called(targetReplicatePChannels) _m.Called(aliveReplicates)
} }
// MockReplicateManagerClient_RemoveOutOfTargetReplicators_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RemoveOutOfTargetReplicators' // MockReplicateManagerClient_RemoveOutdatedReplicators_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RemoveOutdatedReplicators'
type MockReplicateManagerClient_RemoveOutOfTargetReplicators_Call struct { type MockReplicateManagerClient_RemoveOutdatedReplicators_Call struct {
*mock.Call *mock.Call
} }
// RemoveOutOfTargetReplicators is a helper method to define mock.On call // RemoveOutdatedReplicators is a helper method to define mock.On call
// - targetReplicatePChannels []*streamingpb.ReplicatePChannelMeta // - aliveReplicates []*streamingpb.ReplicatePChannelMeta
func (_e *MockReplicateManagerClient_Expecter) RemoveOutOfTargetReplicators(targetReplicatePChannels interface{}) *MockReplicateManagerClient_RemoveOutOfTargetReplicators_Call { func (_e *MockReplicateManagerClient_Expecter) RemoveOutdatedReplicators(aliveReplicates interface{}) *MockReplicateManagerClient_RemoveOutdatedReplicators_Call {
return &MockReplicateManagerClient_RemoveOutOfTargetReplicators_Call{Call: _e.mock.On("RemoveOutOfTargetReplicators", targetReplicatePChannels)} 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) { _c.Call.Run(func(args mock.Arguments) {
run(args[0].([]*streamingpb.ReplicatePChannelMeta)) run(args[0].([]*streamingpb.ReplicatePChannelMeta))
}) })
return _c return _c
} }
func (_c *MockReplicateManagerClient_RemoveOutOfTargetReplicators_Call) Return() *MockReplicateManagerClient_RemoveOutOfTargetReplicators_Call { func (_c *MockReplicateManagerClient_RemoveOutdatedReplicators_Call) Return() *MockReplicateManagerClient_RemoveOutdatedReplicators_Call {
_c.Call.Return() _c.Call.Return()
return _c 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) _c.Run(run)
return _c return _c
} }

View File

@ -23,8 +23,11 @@ type ReplicateManagerClient interface {
// CreateReplicator creates a new replicator for the replicate pchannel. // CreateReplicator creates a new replicator for the replicate pchannel.
CreateReplicator(replicateInfo *streamingpb.ReplicatePChannelMeta) CreateReplicator(replicateInfo *streamingpb.ReplicatePChannelMeta)
// RemoveOutOfTargetReplicators removes replicators that are not in the target replicate pchannels. // RemoveReplicator removes a replicator for the replicate pchannel.
RemoveOutOfTargetReplicators(targetReplicatePChannels []*streamingpb.ReplicatePChannelMeta) RemoveReplicator(replicateInfo *streamingpb.ReplicatePChannelMeta)
// RemoveOutdatedReplicators removes the outdated replicators.
RemoveOutdatedReplicators(aliveReplicates []*streamingpb.ReplicatePChannelMeta)
// Close closes the replicate manager client. // Close closes the replicate manager client.
Close() Close()

View File

@ -20,7 +20,6 @@ import (
"context" "context"
"strings" "strings"
"github.com/samber/lo"
"go.uber.org/zap" "go.uber.org/zap"
"github.com/milvus-io/milvus/internal/metastore/kv/streamingcoord" "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() currentClusterID := paramtable.Get().CommonCfg.ClusterPrefix.GetValue()
if !strings.Contains(replicateInfo.GetSourceChannelName(), currentClusterID) { 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 return
} }
replicatorKey := streamingcoord.BuildReplicatePChannelMetaKey(replicateInfo) replicatorKey := streamingcoord.BuildReplicatePChannelMetaKey(replicateInfo)
@ -69,17 +68,37 @@ func (r *replicateManager) CreateReplicator(replicateInfo *streamingpb.Replicate
logger.Info("created replicator for replicate pchannel") logger.Info("created replicator for replicate pchannel")
} }
func (r *replicateManager) RemoveOutOfTargetReplicators(targetReplicatePChannels []*streamingpb.ReplicatePChannelMeta) { func (r *replicateManager) RemoveReplicator(replicateInfo *streamingpb.ReplicatePChannelMeta) {
targets := lo.KeyBy(targetReplicatePChannels, streamingcoord.BuildReplicatePChannelMetaKey) logger := log.With(
for replicatorKey, replicator := range r.replicators { zap.String("sourceChannel", replicateInfo.GetSourceChannelName()),
if pchannelMeta, ok := targets[replicatorKey]; !ok { 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() replicator.StopReplicate()
delete(r.replicators, replicatorKey) delete(r.replicators, replicatorKey)
delete(r.replicatorPChannels, replicatorKey) delete(r.replicatorPChannels, replicatorKey)
log.Info("removed replicator due to out of target", logger.Info("removed replicator for replicate pchannel")
zap.String("sourceChannel", pchannelMeta.GetSourceChannelName()), }
zap.String("targetChannel", pchannelMeta.GetTargetChannelName()),
) 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 _, ok := alivesMap[replicatorKey]; !ok {
replicator.StopReplicate()
info := r.replicatorPChannels[replicatorKey]
delete(r.replicators, replicatorKey)
delete(r.replicatorPChannels, replicatorKey)
log.Info("removed replicator for replicate pchannel",
zap.String("sourceChannel", info.GetSourceChannelName()),
zap.String("targetChannel", info.GetTargetChannelName()))
} }
} }
} }

View File

@ -19,6 +19,8 @@ package resource
import ( import (
"reflect" "reflect"
clientv3 "go.etcd.io/etcd/client/v3"
"github.com/milvus-io/milvus/internal/cdc/cluster" "github.com/milvus-io/milvus/internal/cdc/cluster"
"github.com/milvus-io/milvus/internal/cdc/controller" "github.com/milvus-io/milvus/internal/cdc/controller"
"github.com/milvus-io/milvus/internal/cdc/replication" "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. // OptReplicateManagerClient provides the replicate manager client to the resource.
func OptReplicateManagerClient(replicateManagerClient replication.ReplicateManagerClient) optResourceInit { func OptReplicateManagerClient(replicateManagerClient replication.ReplicateManagerClient) optResourceInit {
return func(r *resourceImpl) { return func(r *resourceImpl) {
@ -78,6 +87,7 @@ func Init(opts ...optResourceInit) {
newR.clusterClient = cluster.NewClusterClient() newR.clusterClient = cluster.NewClusterClient()
assertNotNil(newR.MetaKV()) assertNotNil(newR.MetaKV())
assertNotNil(newR.ETCD())
assertNotNil(newR.ReplicationCatalog()) assertNotNil(newR.ReplicationCatalog())
assertNotNil(newR.ClusterClient()) assertNotNil(newR.ClusterClient())
assertNotNil(newR.ReplicateManagerClient()) assertNotNil(newR.ReplicateManagerClient())
@ -97,6 +107,7 @@ func Resource() *resourceImpl {
// All utility on it is concurrent-safe and singleton. // All utility on it is concurrent-safe and singleton.
type resourceImpl struct { type resourceImpl struct {
metaKV kv.MetaKv metaKV kv.MetaKv
etcdClient *clientv3.Client
catalog metastore.ReplicationCatalog catalog metastore.ReplicationCatalog
clusterClient cluster.ClusterClient clusterClient cluster.ClusterClient
replicateManagerClient replication.ReplicateManagerClient replicateManagerClient replication.ReplicateManagerClient
@ -108,6 +119,11 @@ func (r *resourceImpl) MetaKV() kv.MetaKv {
return r.metaKV return r.metaKV
} }
// ETCD returns the etcd client.
func (r *resourceImpl) ETCD() *clientv3.Client {
return r.etcdClient
}
// ReplicationCatalog returns the replication catalog. // ReplicationCatalog returns the replication catalog.
func (r *resourceImpl) ReplicationCatalog() metastore.ReplicationCatalog { func (r *resourceImpl) ReplicationCatalog() metastore.ReplicationCatalog {
return r.catalog return r.catalog

View File

@ -6,6 +6,8 @@ package resource
import ( import (
"testing" "testing"
clientv3 "go.etcd.io/etcd/client/v3"
"github.com/milvus-io/milvus/internal/cdc/cluster" "github.com/milvus-io/milvus/internal/cdc/cluster"
"github.com/milvus-io/milvus/internal/cdc/controller" "github.com/milvus-io/milvus/internal/cdc/controller"
"github.com/milvus-io/milvus/internal/cdc/replication" "github.com/milvus-io/milvus/internal/cdc/replication"
@ -22,6 +24,9 @@ func InitForTest(t *testing.T, opts ...optResourceInit) {
if r.metaKV == nil { if r.metaKV == nil {
r.metaKV = mock_kv.NewMockMetaKv(t) r.metaKV = mock_kv.NewMockMetaKv(t)
} }
if r.etcdClient == nil {
r.etcdClient = &clientv3.Client{}
}
if r.catalog == nil { if r.catalog == nil {
r.catalog = mock_metastore.NewMockReplicationCatalog(t) r.catalog = mock_metastore.NewMockReplicationCatalog(t)
} }

View File

@ -19,6 +19,8 @@ package cdc
import ( import (
"context" "context"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/cdc/resource" "github.com/milvus-io/milvus/internal/cdc/resource"
"github.com/milvus-io/milvus/pkg/v2/log" "github.com/milvus-io/milvus/pkg/v2/log"
) )
@ -42,7 +44,11 @@ func (svr *CDCServer) Init() error {
// Start starts CDCServer. // Start starts CDCServer.
func (svr *CDCServer) Start() error { 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") log.Ctx(svr.ctx).Info("CDCServer start successfully")
return nil return nil
} }

View File

@ -32,14 +32,12 @@ import (
"github.com/milvus-io/milvus/internal/cdc/replication/replicatemanager" "github.com/milvus-io/milvus/internal/cdc/replication/replicatemanager"
"github.com/milvus-io/milvus/internal/cdc/resource" "github.com/milvus-io/milvus/internal/cdc/resource"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" 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" "github.com/milvus-io/milvus/internal/util/componentutil"
kvfactory "github.com/milvus-io/milvus/internal/util/dependency/kv" 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/kv"
"github.com/milvus-io/milvus/pkg/v2/log" "github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/util" "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/paramtable"
"github.com/milvus-io/milvus/pkg/v2/util/tikv"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil" "github.com/milvus-io/milvus/pkg/v2/util/typeutil"
) )
@ -48,7 +46,7 @@ type Server struct {
ctx context.Context ctx context.Context
cancel context.CancelFunc cancel context.CancelFunc
metaKV kv.MetaKv watchKV kv.WatchKV
cdcServer *cdc.CDCServer cdcServer *cdc.CDCServer
etcdCli *clientv3.Client etcdCli *clientv3.Client
@ -146,7 +144,8 @@ func (s *Server) init() (err error) {
// Create CDC service. // Create CDC service.
s.cdcServer = cdc.NewCDCServer(s.ctx) s.cdcServer = cdc.NewCDCServer(s.ctx)
resource.Init( resource.Init(
resource.OptMetaKV(s.metaKV), resource.OptMetaKV(s.watchKV),
resource.OptETCD(s.etcdCli),
resource.OptReplicateManagerClient(replicatemanager.NewReplicateManager()), resource.OptReplicateManagerClient(replicatemanager.NewReplicateManager()),
resource.OptController(controllerimpl.NewController()), resource.OptController(controllerimpl.NewController()),
) )
@ -176,18 +175,11 @@ func (s *Server) initMeta() error {
log.Info("cdc connecting to metadata store", zap.String("metaType", metaType)) log.Info("cdc connecting to metadata store", zap.String("metaType", metaType))
metaRootPath := "" metaRootPath := ""
if metaType == util.MetaStoreTypeTiKV { if metaType == util.MetaStoreTypeTiKV {
var err error // TODO: sheep, support watch operation for tikv
s.tikvCli, err = tikv.GetTiKVClient(&paramtable.Get().TiKVCfg) panic("tikv is not supported for cdc")
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)))
} else if metaType == util.MetaStoreTypeEtcd { } else if metaType == util.MetaStoreTypeEtcd {
metaRootPath = params.EtcdCfg.MetaRootPath.GetValue() 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))) etcdkv.WithRequestTimeout(paramtable.Get().ServiceParam.EtcdCfg.RequestTimeout.GetAsDuration(time.Millisecond)))
} }
return nil return nil

View File

@ -8,6 +8,7 @@ packages:
github.com/milvus-io/milvus/pkg/v2/kv: github.com/milvus-io/milvus/pkg/v2/kv:
interfaces: interfaces:
MetaKv: MetaKv:
WatchKV:
github.com/milvus-io/milvus/pkg/v2/streaming/util/message: github.com/milvus-io/milvus/pkg/v2/streaming/util/message:
interfaces: interfaces:
MessageID: MessageID:

File diff suppressed because it is too large Load Diff