diff --git a/Makefile b/Makefile index f4a67188f95..735efa97022 100644 --- a/Makefile +++ b/Makefile @@ -163,6 +163,9 @@ kafka_consumer: storage_consumer: $(GOBUILD) -ldflags '$(LDFLAGS)' -o bin/cdc_storage_consumer ./cmd/storage-consumer/main.go +pulsar_consumer: + $(GOBUILD) -ldflags '$(LDFLAGS)' -o bin/cdc_pulsar_consumer ./cmd/pulsar-consumer/main.go + kafka_consumer_image: @which docker || (echo "docker not found in ${PATH}"; exit 1) DOCKER_BUILDKIT=1 docker build -f ./deployments/ticdc/docker/kafka-consumer.Dockerfile . -t ticdc:kafka-consumer --platform linux/amd64 @@ -171,6 +174,10 @@ storage_consumer_image: @which docker || (echo "docker not found in ${PATH}"; exit 1) DOCKER_BUILDKIT=1 docker build -f ./deployments/ticdc/docker/storage-consumer.Dockerfile . -t ticdc:storage-consumer --platform linux/amd64 +pulsar_consumer_image: + @which docker || (echo "docker not found in ${PATH}"; exit 1) + DOCKER_BUILDKIT=1 docker build -f ./deployments/ticdc/docker/pulsar-consumer.Dockerfile . -t ticdc:pulsar-consumer --platform linux/amd64 + filter_helper: $(GOBUILD) -ldflags '$(LDFLAGS)' -o bin/cdc_filter_helper ./cmd/filter-helper/main.go @@ -215,7 +222,7 @@ check_third_party_binary: @which bin/minio @which bin/bin/schema-registry-start -integration_test_build: check_failpoint_ctl storage_consumer kafka_consumer +integration_test_build: check_failpoint_ctl storage_consumer kafka_consumer pulsar_consumer $(FAILPOINT_ENABLE) $(GOTEST) -ldflags '$(LDFLAGS)' -c -cover -covermode=atomic \ -coverpkg=github.com/pingcap/tiflow/... \ diff --git a/cdc/api/v1/api.go b/cdc/api/v1/api.go index 61e047a5e6f..690008cb023 100644 --- a/cdc/api/v1/api.go +++ b/cdc/api/v1/api.go @@ -308,13 +308,6 @@ func (h *OpenAPI) CreateChangefeed(c *gin.Context) { _ = c.Error(err) return } - - infoStr, err := info.Marshal() - if err != nil { - _ = c.Error(err) - return - } - upstreamInfo := &model.UpstreamInfo{ ID: up.ID, PDEndpoints: strings.Join(up.PdEndpoints, ","), @@ -331,7 +324,9 @@ func (h *OpenAPI) CreateChangefeed(c *gin.Context) { return } - log.Info("Create changefeed successfully!", zap.String("id", changefeedConfig.ID), zap.String("changefeed", infoStr)) + log.Info("Create changefeed successfully!", + zap.String("id", changefeedConfig.ID), + zap.String("changefeed", info.String())) c.Status(http.StatusAccepted) } diff --git a/cdc/api/v1/validator.go b/cdc/api/v1/validator.go index d8f564df40a..b75428d5ca9 100644 --- a/cdc/api/v1/validator.go +++ b/cdc/api/v1/validator.go @@ -120,14 +120,13 @@ func verifyCreateChangefeedConfig( if err != nil { return nil, err } - // set sortEngine and EnableOldValue + // set sortEngine cdcClusterVer, err := version.GetTiCDCClusterVersion(model.ListVersionsFromCaptureInfos(captureInfos)) if err != nil { return nil, err } sortEngine := model.SortUnified - if !cdcClusterVer.ShouldEnableOldValueByDefault() { - replicaConfig.EnableOldValue = false + if !cdcClusterVer.LessThan500RC() { log.Warn("The TiCDC cluster is built from unknown branch or less than 5.0.0-rc, the old-value are disabled by default.") if !cdcClusterVer.ShouldEnableUnifiedSorterByDefault() { sortEngine = model.SortInMemory diff --git a/cdc/api/v2/api_helpers.go b/cdc/api/v2/api_helpers.go index 0e71265dd59..97fa5efa923 100644 --- a/cdc/api/v2/api_helpers.go +++ b/cdc/api/v2/api_helpers.go @@ -381,7 +381,6 @@ func (APIV2HelpersImpl) verifyUpdateChangefeedConfig( if cfg.CertAllowedCN != nil { newUpInfo.CertAllowedCN = cfg.CertAllowedCN } - changefeedInfoChanged := diff.Changed(oldInfo, newInfo) upstreamInfoChanged := diff.Changed(oldUpInfo, newUpInfo) if !changefeedInfoChanged && !upstreamInfoChanged { diff --git a/cdc/api/v2/api_helpers_test.go b/cdc/api/v2/api_helpers_test.go index 51b9608d598..ec4f4a361df 100644 --- a/cdc/api/v2/api_helpers_test.go +++ b/cdc/api/v2/api_helpers_test.go @@ -48,7 +48,6 @@ func TestVerifyCreateChangefeedConfig(t *testing.T) { }) cfg.ReplicaConfig = GetDefaultReplicaConfig() cfg.ReplicaConfig.ForceReplicate = true - cfg.ReplicaConfig.EnableOldValue = false cfg.SinkURI = "mysql://" ctrl.EXPECT().IsChangefeedExists(gomock.Any(), gomock.Any()).Return(false, nil) // disable old value but force replicate, and using mysql sink. @@ -68,47 +67,44 @@ func TestVerifyCreateChangefeedConfig(t *testing.T) { // invalid changefeed id or namespace id cfg.ID = "abdc/sss" - cfInfo, err = h.verifyCreateChangefeedConfig(ctx, cfg, pdClient, ctrl, "en", storage) + _, err = h.verifyCreateChangefeedConfig(ctx, cfg, pdClient, ctrl, "en", storage) require.NotNil(t, err) cfg.ID = "" cfg.Namespace = "abdc/sss" - cfInfo, err = h.verifyCreateChangefeedConfig(ctx, cfg, pdClient, ctrl, "en", storage) + _, err = h.verifyCreateChangefeedConfig(ctx, cfg, pdClient, ctrl, "en", storage) require.NotNil(t, err) cfg.ID = "" cfg.Namespace = "" // changefeed already exists ctrl.EXPECT().IsChangefeedExists(gomock.Any(), gomock.Any()).Return(true, nil) - cfInfo, err = h.verifyCreateChangefeedConfig(ctx, cfg, pdClient, ctrl, "en", storage) + _, err = h.verifyCreateChangefeedConfig(ctx, cfg, pdClient, ctrl, "en", storage) require.NotNil(t, err) ctrl.EXPECT().IsChangefeedExists(gomock.Any(), gomock.Any()).Return(false, cerror.ErrChangeFeedNotExists.GenWithStackByArgs("aaa")) - cfInfo, err = h.verifyCreateChangefeedConfig(ctx, cfg, pdClient, ctrl, "en", storage) + _, err = h.verifyCreateChangefeedConfig(ctx, cfg, pdClient, ctrl, "en", storage) require.Nil(t, err) require.Equal(t, uint64(123), cfInfo.UpstreamID) cfg.TargetTs = 3 cfg.StartTs = 4 ctrl.EXPECT().IsChangefeedExists(gomock.Any(), gomock.Any()).Return(false, nil) - cfInfo, err = h.verifyCreateChangefeedConfig(ctx, cfg, pdClient, ctrl, "en", storage) + _, err = h.verifyCreateChangefeedConfig(ctx, cfg, pdClient, ctrl, "en", storage) require.NotNil(t, err) cfg.TargetTs = 6 - cfg.ReplicaConfig.EnableOldValue = false cfg.SinkURI = "aaab://" ctrl.EXPECT().IsChangefeedExists(gomock.Any(), gomock.Any()).Return(false, nil) - cfInfo, err = h.verifyCreateChangefeedConfig(ctx, cfg, pdClient, ctrl, "en", storage) + _, err = h.verifyCreateChangefeedConfig(ctx, cfg, pdClient, ctrl, "en", storage) require.NotNil(t, err) cfg.SinkURI = string([]byte{0x7f, ' '}) ctrl.EXPECT().IsChangefeedExists(gomock.Any(), gomock.Any()).Return(false, nil) - cfInfo, err = h.verifyCreateChangefeedConfig(ctx, cfg, pdClient, ctrl, "en", storage) + _, err = h.verifyCreateChangefeedConfig(ctx, cfg, pdClient, ctrl, "en", storage) require.NotNil(t, err) cfg.StartTs = 0 // use blackhole to workaround cfg.SinkURI = "blackhole://127.0.0.1:9092/test?protocol=avro" - cfg.ReplicaConfig.EnableOldValue = true cfg.ReplicaConfig.ForceReplicate = false ctrl.EXPECT().IsChangefeedExists(gomock.Any(), gomock.Any()).Return(false, nil) - cfInfo, err = h.verifyCreateChangefeedConfig(ctx, cfg, pdClient, ctrl, "en", storage) + _, err = h.verifyCreateChangefeedConfig(ctx, cfg, pdClient, ctrl, "en", storage) require.NoError(t, err) - require.False(t, cfInfo.Config.EnableOldValue) cfg.ReplicaConfig.ForceReplicate = true ctrl.EXPECT().IsChangefeedExists(gomock.Any(), gomock.Any()).Return(false, nil) @@ -165,16 +161,4 @@ func TestVerifyUpdateChangefeedConfig(t *testing.T) { cfg.TargetTs = 9 newCfInfo, newUpInfo, err = h.verifyUpdateChangefeedConfig(ctx, cfg, oldInfo, oldUpInfo, storage, 0) require.NotNil(t, err) - - cfg.StartTs = 0 - cfg.TargetTs = 0 - cfg.ReplicaConfig.EnableOldValue = true - cfg.SinkURI = "blackhole://127.0.0.1:9092/test?protocol=avro" - newCfInfo, newUpInfo, err = h.verifyUpdateChangefeedConfig(ctx, cfg, oldInfo, oldUpInfo, storage, 0) - require.NoError(t, err) - require.False(t, newCfInfo.Config.EnableOldValue) - - cfg.ReplicaConfig.ForceReplicate = true - newCfInfo, newUpInfo, err = h.verifyUpdateChangefeedConfig(ctx, cfg, oldInfo, oldUpInfo, storage, 0) - require.Error(t, cerror.ErrOldValueNotEnabled, err) } diff --git a/cdc/api/v2/changefeed.go b/cdc/api/v2/changefeed.go index 76539060705..137d1c1e38d 100644 --- a/cdc/api/v2/changefeed.go +++ b/cdc/api/v2/changefeed.go @@ -15,6 +15,7 @@ package v2 import ( "context" + "fmt" "net/http" "sort" "strings" @@ -23,16 +24,17 @@ import ( "github.com/gin-gonic/gin" "github.com/pingcap/errors" "github.com/pingcap/log" + tidbkv "github.com/pingcap/tidb/kv" "github.com/pingcap/tiflow/cdc/api" "github.com/pingcap/tiflow/cdc/capture" "github.com/pingcap/tiflow/cdc/model" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/retry" - "github.com/pingcap/tiflow/pkg/security" "github.com/pingcap/tiflow/pkg/txnutil/gc" "github.com/pingcap/tiflow/pkg/upstream" "github.com/pingcap/tiflow/pkg/util" "github.com/tikv/client-go/v2/oracle" + pd "github.com/tikv/pd/client" clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" ) @@ -135,12 +137,6 @@ func (h *OpenAPIV2) createChangefeed(c *gin.Context) { CAPath: cfg.CAPath, CertAllowedCN: cfg.CertAllowedCN, } - infoStr, err := info.Marshal() - if err != nil { - needRemoveGCSafePoint = true - _ = c.Error(cerror.WrapError(cerror.ErrAPIInvalidParam, err)) - return - } // cannot create changefeed if there are running lightning/restore tasks tlsCfg, err := credential.ToTLSConfig() @@ -176,7 +172,7 @@ func (h *OpenAPIV2) createChangefeed(c *gin.Context) { log.Info("Create changefeed successfully!", zap.String("id", info.ID), - zap.String("changefeed", infoStr)) + zap.String("changefeed", info.String())) c.JSON(http.StatusOK, toAPIModel(info, info.StartTs, info.StartTs, nil, true)) @@ -446,26 +442,31 @@ func (h *OpenAPIV2) updateChangefeed(c *gin.Context) { zap.String("changefeedInfo", oldCfInfo.String()), zap.Any("upstreamInfo", OldUpInfo)) - var pdAddrs []string - var credentials *security.Credential - if OldUpInfo != nil { - pdAddrs = strings.Split(OldUpInfo.PDEndpoints, ",") - credentials = &security.Credential{ - CAPath: OldUpInfo.CAPath, - CertPath: OldUpInfo.CertPath, - KeyPath: OldUpInfo.KeyPath, - CertAllowedCN: OldUpInfo.CertAllowedCN, - } - } - if len(updateCfConfig.PDAddrs) != 0 { - pdAddrs = updateCfConfig.PDAddrs - credentials = updateCfConfig.PDConfig.toCredential() + upManager, err := h.capture.GetUpstreamManager() + if err != nil { + _ = c.Error(err) + return } - storage, err := h.helpers.createTiStore(pdAddrs, credentials) - if err != nil { - _ = c.Error(errors.Trace(err)) + var storage tidbkv.Storage + // if PDAddrs is not empty, use it to create a new kvstore + // Note: upManager is nil in some unit test cases + if len(updateCfConfig.PDAddrs) != 0 || upManager == nil { + pdAddrs := updateCfConfig.PDAddrs + credentials := updateCfConfig.PDConfig.toCredential() + storage, err = h.helpers.createTiStore(pdAddrs, credentials) + if err != nil { + _ = c.Error(errors.Trace(err)) + } + } else { // get the upstream of the changefeed to get the kvstore + up, ok := upManager.Get(oldCfInfo.UpstreamID) + if !ok { + _ = c.Error(errors.New(fmt.Sprintf("upstream %d not found", oldCfInfo.UpstreamID))) + return + } + storage = up.KVStorage } + newCfInfo, newUpInfo, err := h.helpers.verifyUpdateChangefeedConfig(ctx, updateCfConfig, oldCfInfo, OldUpInfo, storage, cfStatus.CheckpointTs) if err != nil { @@ -719,24 +720,26 @@ func (h *OpenAPIV2) resumeChangefeed(c *gin.Context) { return } + var pdClient pd.Client + // if PDAddrs is empty, use the default pdClient if len(cfg.PDAddrs) == 0 { up, err := getCaptureDefaultUpstream(h.capture) if err != nil { _ = c.Error(err) return } - cfg.PDConfig = getUpstreamPDConfig(up) - } - credential := cfg.PDConfig.toCredential() - - timeoutCtx, cancel := context.WithTimeout(ctx, 30*time.Second) - defer cancel() - pdClient, err := h.helpers.getPDClient(timeoutCtx, cfg.PDAddrs, credential) - if err != nil { - _ = c.Error(cerror.WrapError(cerror.ErrAPIInvalidParam, err)) - return + pdClient = up.PDClient + } else { + credential := cfg.PDConfig.toCredential() + timeoutCtx, cancel := context.WithTimeout(ctx, 30*time.Second) + defer cancel() + pdClient, err = h.helpers.getPDClient(timeoutCtx, cfg.PDAddrs, credential) + if err != nil { + _ = c.Error(cerror.WrapError(cerror.ErrAPIInvalidParam, err)) + return + } + defer pdClient.Close() } - defer pdClient.Close() if err := h.helpers.verifyResumeChangefeedConfig( ctx, diff --git a/cdc/api/v2/changefeed_test.go b/cdc/api/v2/changefeed_test.go index 8bfcc12eb9d..f85221a2f09 100644 --- a/cdc/api/v2/changefeed_test.go +++ b/cdc/api/v2/changefeed_test.go @@ -333,14 +333,14 @@ func TestUpdateChangefeed(t *testing.T) { t.Parallel() update := testCase{url: "/api/v2/changefeeds/%s", method: "PUT"} helpers := NewMockAPIV2Helpers(gomock.NewController(t)) - cp := mock_capture.NewMockCapture(gomock.NewController(t)) - apiV2 := NewOpenAPIV2ForTest(cp, helpers) + mockCapture := mock_capture.NewMockCapture(gomock.NewController(t)) + apiV2 := NewOpenAPIV2ForTest(mockCapture, helpers) router := newRouter(apiV2) statusProvider := &mockStatusProvider{} - cp.EXPECT().StatusProvider().Return(statusProvider).AnyTimes() - cp.EXPECT().IsReady().Return(true).AnyTimes() - cp.EXPECT().IsController().Return(true).AnyTimes() + mockCapture.EXPECT().StatusProvider().Return(statusProvider).AnyTimes() + mockCapture.EXPECT().IsReady().Return(true).AnyTimes() + mockCapture.EXPECT().IsController().Return(true).AnyTimes() // case 1 invalid id invalidID := "Invalid_#" @@ -394,7 +394,7 @@ func TestUpdateChangefeed(t *testing.T) { etcdClient.EXPECT(). GetUpstreamInfo(gomock.Any(), gomock.Eq(uint64(100)), gomock.Any()). Return(nil, cerrors.ErrUpstreamNotFound).Times(1) - cp.EXPECT().GetEtcdClient().Return(etcdClient).AnyTimes() + mockCapture.EXPECT().GetEtcdClient().Return(etcdClient).AnyTimes() w = httptest.NewRecorder() req, _ = http.NewRequestWithContext(context.Background(), update.method, @@ -411,7 +411,7 @@ func TestUpdateChangefeed(t *testing.T) { etcdClient.EXPECT(). GetUpstreamInfo(gomock.Any(), gomock.Eq(uint64(1)), gomock.Any()). Return(nil, nil).AnyTimes() - cp.EXPECT().GetEtcdClient().Return(etcdClient).AnyTimes() + mockCapture.EXPECT().GetEtcdClient().Return(etcdClient).AnyTimes() w = httptest.NewRecorder() req, _ = http.NewRequestWithContext(context.Background(), update.method, @@ -448,6 +448,7 @@ func TestUpdateChangefeed(t *testing.T) { createTiStore(gomock.Any(), gomock.Any()). Return(nil, nil). AnyTimes() + mockCapture.EXPECT().GetUpstreamManager().Return(nil, nil).AnyTimes() helpers.EXPECT(). verifyUpdateChangefeedConfig(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). Return(&model.ChangeFeedInfo{}, &model.UpstreamInfo{}, cerrors.ErrChangefeedUpdateRefused). @@ -467,6 +468,7 @@ func TestUpdateChangefeed(t *testing.T) { require.Equal(t, http.StatusBadRequest, w.Code) // case 7: update transaction failed + mockCapture.EXPECT().GetUpstreamManager().Return(upstream.NewManager4Test(&mockPDClient{}), nil).AnyTimes() helpers.EXPECT(). verifyUpdateChangefeedConfig(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). Return(&model.ChangeFeedInfo{}, &model.UpstreamInfo{}, nil). @@ -490,6 +492,7 @@ func TestUpdateChangefeed(t *testing.T) { verifyUpdateChangefeedConfig(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). Return(oldCfInfo, &model.UpstreamInfo{}, nil). Times(1) + mockCapture.EXPECT().GetUpstreamManager().Return(upstream.NewManager4Test(&mockPDClient{}), nil).AnyTimes() etcdClient.EXPECT(). UpdateChangefeedAndUpstream(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). Return(nil).Times(1) @@ -506,6 +509,7 @@ func TestUpdateChangefeed(t *testing.T) { verifyUpdateChangefeedConfig(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). Return(oldCfInfo, &model.UpstreamInfo{}, nil). Times(1) + mockCapture.EXPECT().GetUpstreamManager().Return(upstream.NewManager4Test(&mockPDClient{}), nil).AnyTimes() etcdClient.EXPECT(). UpdateChangefeedAndUpstream(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). Return(nil).Times(1) diff --git a/cdc/api/v2/model.go b/cdc/api/v2/model.go index c9e9eeddfa9..e4da00c4928 100644 --- a/cdc/api/v2/model.go +++ b/cdc/api/v2/model.go @@ -177,7 +177,6 @@ func (d *JSONDuration) UnmarshalJSON(b []byte) error { type ReplicaConfig struct { MemoryQuota uint64 `json:"memory_quota"` CaseSensitive bool `json:"case_sensitive"` - EnableOldValue bool `json:"enable_old_value"` ForceReplicate bool `json:"force_replicate"` IgnoreIneligibleTable bool `json:"ignore_ineligible_table"` CheckGCSafePoint bool `json:"check_gc_safe_point"` @@ -206,7 +205,6 @@ func (c *ReplicaConfig) toInternalReplicaConfigWithOriginConfig( ) *config.ReplicaConfig { res.MemoryQuota = c.MemoryQuota res.CaseSensitive = c.CaseSensitive - res.EnableOldValue = c.EnableOldValue res.ForceReplicate = c.ForceReplicate res.CheckGCSafePoint = c.CheckGCSafePoint res.EnableSyncPoint = c.EnableSyncPoint @@ -487,7 +485,6 @@ func ToAPIReplicaConfig(c *config.ReplicaConfig) *ReplicaConfig { res := &ReplicaConfig{ MemoryQuota: cloned.MemoryQuota, CaseSensitive: cloned.CaseSensitive, - EnableOldValue: cloned.EnableOldValue, ForceReplicate: cloned.ForceReplicate, IgnoreIneligibleTable: cloned.IgnoreIneligibleTable, CheckGCSafePoint: cloned.CheckGCSafePoint, diff --git a/cdc/api/v2/model_test.go b/cdc/api/v2/model_test.go index 3b827345d3b..e524e3e54ba 100644 --- a/cdc/api/v2/model_test.go +++ b/cdc/api/v2/model_test.go @@ -32,7 +32,6 @@ import ( var defaultAPIConfig = &ReplicaConfig{ MemoryQuota: config.DefaultChangefeedMemoryQuota, CaseSensitive: true, - EnableOldValue: true, CheckGCSafePoint: true, BDRMode: util.AddressOf(false), EnableSyncPoint: util.AddressOf(false), @@ -51,7 +50,7 @@ var defaultAPIConfig = &ReplicaConfig{ NullString: config.NULL, BinaryEncodingMethod: config.BinaryEncodingBase64, }, - EncoderConcurrency: util.AddressOf(16), + EncoderConcurrency: util.AddressOf(config.DefaultEncoderGroupConcurrency), Terminator: util.AddressOf(config.CRLF), DateSeparator: util.AddressOf(config.DateSeparatorDay.String()), EnablePartitionSeparator: util.AddressOf(true), @@ -95,7 +94,6 @@ func TestDefaultReplicaConfig(t *testing.T) { func TestToAPIReplicaConfig(t *testing.T) { cfg := config.GetDefaultReplicaConfig() - cfg.EnableOldValue = false cfg.CheckGCSafePoint = false cfg.Sink = &config.SinkConfig{ DispatchRules: []*config.DispatchRule{ diff --git a/cdc/capture/capture.go b/cdc/capture/capture.go index 8a3c0817272..b1b2fda90e7 100644 --- a/cdc/capture/capture.go +++ b/cdc/capture/capture.go @@ -40,6 +40,7 @@ import ( "github.com/pingcap/tiflow/pkg/upstream" "github.com/pingcap/tiflow/pkg/util" "github.com/pingcap/tiflow/pkg/version" + pd "github.com/tikv/pd/client" "go.etcd.io/etcd/client/v3/concurrency" "go.etcd.io/etcd/server/v3/mvcc" "go.uber.org/zap" @@ -81,6 +82,7 @@ type captureImpl struct { liveness model.Liveness config *config.ServerConfig + pdClient pd.Client pdEndpoints []string ownerMu sync.Mutex owner owner.Owner @@ -128,6 +130,7 @@ func NewCapture(pdEndpoints []string, etcdClient etcd.CDCEtcdClient, grpcService *p2p.ServerWrapper, sortEngineMangerFactory *factory.SortEngineFactory, + pdClient pd.Client, ) Capture { conf := config.GetGlobalServerConfig() return &captureImpl{ @@ -142,8 +145,8 @@ func NewCapture(pdEndpoints []string, newController: controller.NewController, info: &model.CaptureInfo{}, sortEngineFactory: sortEngineMangerFactory, - - migrator: migrate.NewMigrator(etcdClient, pdEndpoints, conf), + migrator: migrate.NewMigrator(etcdClient, pdEndpoints, conf), + pdClient: pdClient, } } @@ -227,7 +230,7 @@ func (c *captureImpl) reset(ctx context.Context) error { c.upstreamManager.Close() } c.upstreamManager = upstream.NewManager(ctx, c.EtcdClient.GetGCServiceID()) - _, err = c.upstreamManager.AddDefaultUpstream(c.pdEndpoints, c.config.Security) + _, err = c.upstreamManager.AddDefaultUpstream(c.pdEndpoints, c.config.Security, c.pdClient) if err != nil { return errors.Trace(err) } diff --git a/cdc/entry/mounter.go b/cdc/entry/mounter.go index 225540bdc25..57316aae6b9 100644 --- a/cdc/entry/mounter.go +++ b/cdc/entry/mounter.go @@ -496,11 +496,6 @@ func (m *mounter) mountRowKVEntry(tableInfo *model.TableInfo, row *rowKVEntry, d preRawCols []types.Datum preChecksum uint32 ) - // Since we now always use old value internally, - // we need to control the output(sink will use the PreColumns field to determine whether to output old value). - // Normally old value is output when only enableOldValue is on, - // but for the Delete event, when the old value feature is off, - // the HandleKey column needs to be included as well. So we need to do the following filtering. if row.PreRowExist { // FIXME(leoppro): using pre table info to mounter pre column datum // the pre column and current column in one event may using different table info diff --git a/cdc/model/changefeed.go b/cdc/model/changefeed.go index b910a3894a9..a91a86aa5d4 100644 --- a/cdc/model/changefeed.go +++ b/cdc/model/changefeed.go @@ -18,7 +18,6 @@ import ( "math" "net/url" "regexp" - "strings" "time" "github.com/pingcap/errors" @@ -240,9 +239,9 @@ func (info *ChangeFeedInfo) String() (str string) { return } - clone.SinkURI, err = util.MaskSinkURI(clone.SinkURI) - if err != nil { - log.Error("failed to marshal changefeed info", zap.Error(err)) + clone.SinkURI = util.MaskSensitiveDataInURI(clone.SinkURI) + if clone.Config != nil { + clone.Config.MaskSensitiveData() } str, err = clone.Marshal() @@ -373,6 +372,9 @@ func (info *ChangeFeedInfo) RmUnusedFields() { } func (info *ChangeFeedInfo) rmMQOnlyFields() { + log.Info("since the downstream is not a MQ, remove MQ only fields", + zap.String("namespace", info.Namespace), + zap.String("changefeed", info.ID)) info.Config.Sink.DispatchRules = nil info.Config.Sink.SchemaRegistry = nil info.Config.Sink.EncoderConcurrency = nil @@ -431,14 +433,6 @@ func (info *ChangeFeedInfo) FixIncompatible() { inheritV66 := creatorVersionGate.ChangefeedInheritSchedulerConfigFromV66() info.fixScheduler(inheritV66) log.Info("Fix incompatible scheduler completed", zap.String("changefeed", info.String())) - - if creatorVersionGate.ChangefeedAdjustEnableOldValueByProtocol() { - log.Info("Start fixing incompatible enable old value", zap.String("changefeed", info.String()), - zap.Bool("enableOldValue", info.Config.EnableOldValue)) - info.fixEnableOldValue() - log.Info("Fix incompatible enable old value completed", zap.String("changefeed", info.String()), - zap.Bool("enableOldValue", info.Config.EnableOldValue)) - } } // fixState attempts to fix state loss from upgrading the old owner to the new owner. @@ -509,18 +503,6 @@ func (info *ChangeFeedInfo) fixMySQLSinkProtocol() { } } -func (info *ChangeFeedInfo) fixEnableOldValue() { - uri, err := url.Parse(info.SinkURI) - if err != nil { - // this is impossible to happen, since the changefeed registered successfully. - log.Warn("parse sink URI failed", zap.Error(err)) - return - } - scheme := strings.ToLower(uri.Scheme) - protocol := uri.Query().Get(config.ProtocolKey) - info.Config.AdjustEnableOldValue(scheme, protocol) -} - func (info *ChangeFeedInfo) fixMQSinkProtocol() { uri, err := url.Parse(info.SinkURI) if err != nil { @@ -561,11 +543,11 @@ func (info *ChangeFeedInfo) fixMQSinkProtocol() { } func (info *ChangeFeedInfo) updateSinkURIAndConfigProtocol(uri *url.URL, newProtocol string, newQuery url.Values) { - oldRawQuery := uri.RawQuery newRawQuery := newQuery.Encode() + maskedURI, _ := util.MaskSinkURI(uri.String()) log.Info("handle incompatible protocol from sink URI", - zap.String("oldUriQuery", oldRawQuery), - zap.String("fixedUriQuery", newQuery.Encode())) + zap.String("oldURI", maskedURI), + zap.String("newProtocol", newProtocol)) uri.RawQuery = newRawQuery fixedSinkURI := uri.String() diff --git a/cdc/model/changefeed_test.go b/cdc/model/changefeed_test.go index 40380e8800a..318c0242f16 100644 --- a/cdc/model/changefeed_test.go +++ b/cdc/model/changefeed_test.go @@ -248,7 +248,6 @@ func TestVerifyAndComplete(t *testing.T) { Config: &config.ReplicaConfig{ MemoryQuota: 1073741824, CaseSensitive: true, - EnableOldValue: true, CheckGCSafePoint: true, SyncPointInterval: util.AddressOf(time.Minute * 10), SyncPointRetention: util.AddressOf(time.Hour * 24), @@ -923,7 +922,6 @@ func TestChangeFeedInfoClone(t *testing.T) { StartTs: 417257993615179777, Config: &config.ReplicaConfig{ CaseSensitive: true, - EnableOldValue: true, CheckGCSafePoint: true, }, } @@ -932,11 +930,8 @@ func TestChangeFeedInfoClone(t *testing.T) { require.Nil(t, err) sinkURI := "mysql://unix:/var/run/tidb.sock" cloned.SinkURI = sinkURI - cloned.Config.EnableOldValue = false require.Equal(t, sinkURI, cloned.SinkURI) - require.False(t, cloned.Config.EnableOldValue) require.Equal(t, "blackhole://", info.SinkURI) - require.True(t, info.Config.EnableOldValue) } func TestChangefeedInfoStringer(t *testing.T) { diff --git a/cdc/model/sink.go b/cdc/model/sink.go index fbbc30eb579..decb3d1e8cc 100644 --- a/cdc/model/sink.go +++ b/cdc/model/sink.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/integrity" "github.com/pingcap/tiflow/pkg/quotes" + "github.com/pingcap/tiflow/pkg/sink" "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" ) @@ -271,7 +272,7 @@ func (r *RedoLog) GetCommitTs() Ts { } // TrySplitAndSortUpdateEvent redo log do nothing -func (r *RedoLog) TrySplitAndSortUpdateEvent() error { +func (r *RedoLog) TrySplitAndSortUpdateEvent(_ string) error { return nil } @@ -379,7 +380,7 @@ func (r *RowChangedEvent) GetCommitTs() uint64 { } // TrySplitAndSortUpdateEvent do nothing -func (r *RowChangedEvent) TrySplitAndSortUpdateEvent() error { +func (r *RowChangedEvent) TrySplitAndSortUpdateEvent(_ string) error { return nil } @@ -767,8 +768,8 @@ func (t *SingleTableTxn) GetCommitTs() uint64 { } // TrySplitAndSortUpdateEvent split update events if unique key is updated -func (t *SingleTableTxn) TrySplitAndSortUpdateEvent() error { - if len(t.Rows) < 2 { +func (t *SingleTableTxn) TrySplitAndSortUpdateEvent(scheme string) error { + if !t.shouldSplitUpdateEvent(scheme) { return nil } newRows, err := trySplitAndSortUpdateEvent(t.Rows) @@ -779,6 +780,22 @@ func (t *SingleTableTxn) TrySplitAndSortUpdateEvent() error { return nil } +// Whether split a single update event into delete and insert events? +// +// For the MySQL Sink, there is no need to split a single unique key changed update event, this +// is also to keep the backward compatibility, the same behavior as before. +// +// For the Kafka and Storage sink, always split a single unique key changed update event, since: +// 1. Avro and CSV does not output the previous column values for the update event, so it would +// cause consumer missing data if the unique key changed event is not split. +// 2. Index-Value Dispatcher cannot work correctly if the unique key changed event isn't split. +func (t *SingleTableTxn) shouldSplitUpdateEvent(sinkScheme string) bool { + if len(t.Rows) < 2 && sink.IsMySQLCompatibleScheme(sinkScheme) { + return false + } + return true +} + // trySplitAndSortUpdateEvent try to split update events if unique key is updated // returns true if some updated events is split func trySplitAndSortUpdateEvent( diff --git a/cdc/model/sink_test.go b/cdc/model/sink_test.go index e0a0aed9d14..eb899d6c109 100644 --- a/cdc/model/sink_test.go +++ b/cdc/model/sink_test.go @@ -20,6 +20,7 @@ import ( timodel "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/types" + "github.com/pingcap/tiflow/pkg/sink" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -563,3 +564,48 @@ func TestTrySplitAndSortUpdateEvent(t *testing.T) { require.NoError(t, err) require.Equal(t, 1, len(result)) } + +var ukUpdatedEvent = &RowChangedEvent{ + PreColumns: []*Column{ + { + Name: "col1", + Flag: BinaryFlag, + Value: "col1-value", + }, + { + Name: "col2", + Flag: HandleKeyFlag | UniqueKeyFlag, + Value: "col2-value", + }, + }, + + Columns: []*Column{ + { + Name: "col1", + Flag: BinaryFlag, + Value: "col1-value", + }, + { + Name: "col2", + Flag: HandleKeyFlag | UniqueKeyFlag, + Value: "col2-value-updated", + }, + }, +} + +func TestTrySplitAndSortUpdateEventOne(t *testing.T) { + txn := &SingleTableTxn{ + Rows: []*RowChangedEvent{ukUpdatedEvent}, + } + + err := txn.TrySplitAndSortUpdateEvent(sink.KafkaScheme) + require.NoError(t, err) + require.Len(t, txn.Rows, 2) + + txn = &SingleTableTxn{ + Rows: []*RowChangedEvent{ukUpdatedEvent}, + } + err = txn.TrySplitAndSortUpdateEvent(sink.MySQLScheme) + require.NoError(t, err) + require.Len(t, txn.Rows, 1) +} diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go index 336c88a2ec5..c93d83baae2 100644 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -650,7 +650,7 @@ LOOP2: zap.Uint64("changefeedEpoch", epoch), zap.Uint64("checkpointTs", checkpointTs), zap.Uint64("resolvedTs", c.resolvedTs), - zap.Stringer("info", c.state.Info)) + zap.String("info", c.state.Info.String())) return nil } @@ -726,7 +726,7 @@ func (c *changefeed) releaseResources(ctx cdcContext.Context) { zap.String("namespace", c.id.Namespace), zap.String("changefeed", c.id.ID), zap.Any("status", c.state.Status), - zap.Stringer("info", c.state.Info), + zap.String("info", c.state.Info.String()), zap.Bool("isRemoved", c.isRemoved)) } diff --git a/cdc/owner/feed_state_manager.go b/cdc/owner/feed_state_manager.go index bf6bf0b9d78..2d33be7bdbd 100644 --- a/cdc/owner/feed_state_manager.go +++ b/cdc/owner/feed_state_manager.go @@ -39,11 +39,6 @@ const ( defaultBackoffMaxElapsedTime = 30 * time.Minute defaultBackoffRandomizationFactor = 0.1 defaultBackoffMultiplier = 2.0 - - // If all states recorded in window are 'normal', it can be assumed that the changefeed - // is running steady. And then if we enter a state other than normal at next tick, - // the backoff must be reset. - defaultStateWindowSize = 512 ) // feedStateManager manages the ReactorState of a changefeed @@ -59,7 +54,7 @@ type feedStateManager struct { shouldBeRemoved bool adminJobQueue []*model.AdminJob - stateHistory [defaultStateWindowSize]model.FeedState + isRetrying bool lastErrorRetryTime time.Time // time of last error for a changefeed lastErrorRetryCheckpointTs model.Ts // checkpoint ts of last retry lastWarningReportCheckpointTs model.Ts // checkpoint ts of last warning report @@ -99,26 +94,6 @@ func (m *feedStateManager) resetErrRetry() { m.lastErrorRetryTime = time.Unix(0, 0) } -// isChangefeedStable check if there are states other than 'normal' in this sliding window. -func (m *feedStateManager) isChangefeedStable() bool { - for _, val := range m.stateHistory { - if val != model.StateNormal { - return false - } - } - - return true -} - -// shiftStateWindow shift the sliding window -func (m *feedStateManager) shiftStateWindow(state model.FeedState) { - for i := 0; i < defaultStateWindowSize-1; i++ { - m.stateHistory[i] = m.stateHistory[i+1] - } - - m.stateHistory[defaultStateWindowSize-1] = state -} - func (m *feedStateManager) Tick( state *orchestrator.ChangefeedReactorState, resolvedTs model.Ts, @@ -134,7 +109,6 @@ func (m *feedStateManager) Tick( } } - m.shiftStateWindow(state.Info.State) m.checkAndInitLastRetryCheckpointTs(state.Status) m.state = state @@ -175,13 +149,13 @@ func (m *feedStateManager) Tick( // NextBackOff() will return -1 once the MaxElapsedTime has elapsed, // set the changefeed to failed state. if m.backoffInterval == m.errBackoff.Stop { - log.Warn("The changefeed won't be restarted "+ - "as it has been experiencing failures for "+ + log.Error("The changefeed won't be restarted as it has been experiencing failures for "+ "an extended duration", - zap.Duration( - "maxElapsedTime", - m.errBackoff.MaxElapsedTime, - ), + zap.Duration("maxElapsedTime", m.errBackoff.MaxElapsedTime), + zap.String("namespace", m.state.ID.Namespace), + zap.String("changefeed", m.state.ID.ID), + zap.Time("lastRetryTime", m.lastErrorRetryTime), + zap.Uint64("lastRetryCheckpointTs", m.lastErrorRetryCheckpointTs), ) m.shouldBeRunning = false m.patchState(model.StateFailed) @@ -572,13 +546,10 @@ func (m *feedStateManager) handleError(errs ...*model.RunningError) { }) } - // If we enter into an abnormal state 'pending' for this changefeed now - // but haven't seen abnormal states in a sliding window (512 ticks), - // it can be assumed that this changefeed meets a sudden change from a stable condition. - // So we can reset the exponential backoff and re-backoff from the InitialInterval. - // TODO: this detection policy should be added into unit test. - if m.isChangefeedStable() { + // The errBackoff needs to be reset before the first retry. + if !m.isRetrying { m.resetErrRetry() + m.isRetrying = true } } @@ -653,6 +624,7 @@ func (m *feedStateManager) checkAndChangeState() { zap.Uint64("checkpointTs", m.state.Status.CheckpointTs), zap.Uint64("lastRetryCheckpointTs", m.lastErrorRetryCheckpointTs)) m.patchState(model.StateNormal) + m.isRetrying = false } } diff --git a/cdc/owner/feed_state_manager_test.go b/cdc/owner/feed_state_manager_test.go index 3479e3c6e82..fa48e5be9e8 100644 --- a/cdc/owner/feed_state_manager_test.go +++ b/cdc/owner/feed_state_manager_test.go @@ -47,9 +47,7 @@ func (p *mockPD) GetTS(_ context.Context) (int64, int64, error) { // newFeedStateManager4Test creates feedStateManager for test func newFeedStateManager4Test( - initialIntervalInMs time.Duration, - maxIntervalInMs time.Duration, - maxElapsedTimeInMs time.Duration, + initialIntervalInMs, maxIntervalInMs, maxElapsedTimeInMs int, multiplier float64, ) *feedStateManager { f := new(feedStateManager) @@ -58,9 +56,9 @@ func newFeedStateManager4Test( f.upstream.PDClock = pdutil.NewClock4Test() f.errBackoff = backoff.NewExponentialBackOff() - f.errBackoff.InitialInterval = initialIntervalInMs * time.Millisecond - f.errBackoff.MaxInterval = maxIntervalInMs * time.Millisecond - f.errBackoff.MaxElapsedTime = maxElapsedTimeInMs * time.Millisecond + f.errBackoff.InitialInterval = time.Duration(initialIntervalInMs) * time.Millisecond + f.errBackoff.MaxInterval = time.Duration(maxIntervalInMs) * time.Millisecond + f.errBackoff.MaxElapsedTime = time.Duration(maxElapsedTimeInMs) * time.Millisecond f.errBackoff.Multiplier = multiplier f.errBackoff.RandomizationFactor = 0 @@ -426,7 +424,6 @@ func TestChangefeedStatusNotExist(t *testing.T) { "sort-engine": "unified", "config": { "case-sensitive": true, - "enable-old-value": true, "force-replicate": false, "check-gc-safe-point": true, "filter": { @@ -691,7 +688,7 @@ func TestBackoffNeverStops(t *testing.T) { func TestUpdateChangefeedEpoch(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) // Set a long backoff time - manager := newFeedStateManager4Test(time.Hour, time.Hour, 0, 1.0) + manager := newFeedStateManager4Test(int(time.Hour), int(time.Hour), 0, 1.0) state := orchestrator.NewChangefeedReactorState(etcd.DefaultCDCClusterID, ctx.ChangefeedVars().ID) tester := orchestrator.NewReactorStateTester(t, state, nil) @@ -850,3 +847,90 @@ func TestHandleWarning(t *testing.T) { require.Equal(t, model.StateFailed, state.Info.State) require.False(t, manager.ShouldRunning()) } + +func TestErrorAfterWarning(t *testing.T) { + t.Parallel() + + maxElapsedTimeInMs := 2000 + ctx := cdcContext.NewBackendContext4Test(true) + manager := newFeedStateManager4Test(200, 1600, maxElapsedTimeInMs, 2.0) + state := orchestrator.NewChangefeedReactorState(etcd.DefaultCDCClusterID, + ctx.ChangefeedVars().ID) + tester := orchestrator.NewReactorStateTester(t, state, nil) + state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { + require.Nil(t, info) + return &model.ChangeFeedInfo{SinkURI: "123", Config: &config.ReplicaConfig{}}, true, nil + }) + state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { + require.Nil(t, status) + return &model.ChangeFeedStatus{ + CheckpointTs: 200, + }, true, nil + }) + + tester.MustApplyPatches() + manager.Tick(state, 0) + tester.MustApplyPatches() + require.Equal(t, model.StateNormal, state.Info.State) + require.True(t, manager.ShouldRunning()) + + // 1. test when an warning occurs, the changefeed state will be changed to warning + // and it will still keep running + state.PatchTaskPosition(ctx.GlobalVars().CaptureInfo.ID, + func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { + return &model.TaskPosition{Warning: &model.RunningError{ + Addr: ctx.GlobalVars().CaptureInfo.AdvertiseAddr, + Code: "[CDC:ErrSinkManagerRunError]", // it is fake error + Message: "fake error for test", + }}, true, nil + }) + tester.MustApplyPatches() + manager.Tick(state, 0) + // some patches will be generated when the manager.Tick is called + // so we need to apply the patches before we check the state + tester.MustApplyPatches() + require.Equal(t, model.StateWarning, state.Info.State) + require.True(t, manager.ShouldRunning()) + + // 2. test when the changefeed is in warning state, and the checkpointTs is not progressing, + // the changefeed state will remain warning + state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { + require.NotNil(t, status) + return &model.ChangeFeedStatus{ + CheckpointTs: 200, + }, true, nil + }) + tester.MustApplyPatches() + manager.Tick(state, 0) + tester.MustApplyPatches() + require.Equal(t, model.StateWarning, state.Info.State) + require.True(t, manager.ShouldRunning()) + + // 3. Sleep maxElapsedTimeInMs to wait backoff timeout. And when an error occurs after an warning, + // the backoff will be reseted, and changefeed state will be changed to warning and it will still + // keep running. + time.Sleep(time.Millisecond * time.Duration(maxElapsedTimeInMs)) + state.PatchTaskPosition(ctx.GlobalVars().CaptureInfo.ID, + func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { + return &model.TaskPosition{Error: &model.RunningError{ + Addr: ctx.GlobalVars().CaptureInfo.AdvertiseAddr, + Code: "[CDC:ErrSinkManagerRunError]", // it is fake error + Message: "fake error for test", + }}, true, nil + }) + tester.MustApplyPatches() + + manager.Tick(state, 0) + // some patches will be generated when the manager.Tick is called + // so we need to apply the patches before we check the state + tester.MustApplyPatches() + require.Equal(t, model.StatePending, state.Info.State) + require.False(t, manager.ShouldRunning()) + manager.Tick(state, 0) + + // some patches will be generated when the manager.Tick is called + // so we need to apply the patches before we check the state + tester.MustApplyPatches() + require.Equal(t, model.StateWarning, state.Info.State) + require.True(t, manager.ShouldRunning()) +} diff --git a/cdc/owner/metrics.go b/cdc/owner/metrics.go index 97b7c43a46b..6ea7a084c75 100644 --- a/cdc/owner/metrics.go +++ b/cdc/owner/metrics.go @@ -159,6 +159,6 @@ func InitMetrics(registry *prometheus.Registry) { func lagBucket() []float64 { buckets := prometheus.LinearBuckets(0.5, 0.5, 20) buckets = append(buckets, prometheus.LinearBuckets(11, 1, 10)...) - buckets = append(buckets, prometheus.ExponentialBuckets(40, 2, 10)...) + buckets = append(buckets, prometheus.ExponentialBuckets(21, 2, 10)...) return buckets } diff --git a/cdc/owner/metrics_test.go b/cdc/owner/metrics_test.go index b927d3f7497..f45c669bbe7 100644 --- a/cdc/owner/metrics_test.go +++ b/cdc/owner/metrics_test.go @@ -23,5 +23,7 @@ func TestLagBucket(t *testing.T) { buckets := lagBucket() require.Equal(t, 40, len(buckets)) require.Equal(t, 0.5, buckets[0]) - require.Equal(t, float64(20480), buckets[39]) + require.Equal(t, 1.0, buckets[1]) + require.Equal(t, 21.0, buckets[30]) + require.Equal(t, float64(10752), buckets[39]) } diff --git a/cdc/processor/processor_test.go b/cdc/processor/processor_test.go index cd259de8821..976613ab8d0 100644 --- a/cdc/processor/processor_test.go +++ b/cdc/processor/processor_test.go @@ -120,7 +120,6 @@ func initProcessor4Test( "sort-dir": ".", "config": { "case-sensitive": true, - "enable-old-value": false, "force-replicate": false, "check-gc-safe-point": true, "filter": { diff --git a/cdc/processor/sinkmanager/manager.go b/cdc/processor/sinkmanager/manager.go index f3842e5af04..74c5513888c 100644 --- a/cdc/processor/sinkmanager/manager.go +++ b/cdc/processor/sinkmanager/manager.go @@ -194,7 +194,6 @@ func (m *SinkManager) Run(ctx context.Context, warnings ...chan<- error) (err er }() splitTxn := util.GetOrZero(m.changefeedInfo.Config.Sink.TxnAtomicity).ShouldSplitTxn() - enableOldValue := m.changefeedInfo.Config.EnableOldValue gcErrors := make(chan error, 16) sinkErrors := make(chan error, 16) @@ -204,7 +203,7 @@ func (m *SinkManager) Run(ctx context.Context, warnings ...chan<- error) (err er if m.sinkEg == nil { var sinkCtx context.Context m.sinkEg, sinkCtx = errgroup.WithContext(m.managerCtx) - m.startSinkWorkers(sinkCtx, m.sinkEg, splitTxn, enableOldValue) + m.startSinkWorkers(sinkCtx, m.sinkEg, splitTxn) m.sinkEg.Go(func() error { return m.generateSinkTasks(sinkCtx) }) m.wg.Add(1) go func() { @@ -224,7 +223,7 @@ func (m *SinkManager) Run(ctx context.Context, warnings ...chan<- error) (err er if m.redoDMLMgr != nil && m.redoEg == nil { var redoCtx context.Context m.redoEg, redoCtx = errgroup.WithContext(m.managerCtx) - m.startRedoWorkers(redoCtx, m.redoEg, enableOldValue) + m.startRedoWorkers(redoCtx, m.redoEg) m.redoEg.Go(func() error { return m.generateRedoTasks(redoCtx) }) m.wg.Add(1) go func() { @@ -269,6 +268,7 @@ func (m *SinkManager) Run(ctx context.Context, warnings ...chan<- error) (err er zap.Error(err)) m.clearSinkFactory() + // To release memory quota ASAP, close all table sinks manually. start := time.Now() log.Info("Sink manager is closing all table sinks", zap.String("namespace", m.changefeedID.Namespace), @@ -305,6 +305,12 @@ func (m *SinkManager) Run(ctx context.Context, warnings ...chan<- error) (err er } } +func (m *SinkManager) needsStuckCheck() bool { + m.sinkFactory.Lock() + defer m.sinkFactory.Unlock() + return m.sinkFactory.f != nil && m.sinkFactory.f.Category() == factory.CategoryMQ +} + func (m *SinkManager) initSinkFactory() (chan error, uint64) { m.sinkFactory.Lock() defer m.sinkFactory.Unlock() @@ -372,38 +378,33 @@ func (m *SinkManager) clearSinkFactory() { } } -func (m *SinkManager) putSinkFactoryError(err error, version uint64) { +func (m *SinkManager) putSinkFactoryError(err error, version uint64) (success bool) { m.sinkFactory.Lock() defer m.sinkFactory.Unlock() - skipped := true if version == m.sinkFactory.version { select { case m.sinkFactory.errors <- err: - skipped = false default: } + return true } - log.Info("Sink manager tries to put an sink error", - zap.String("namespace", m.changefeedID.Namespace), - zap.String("changefeed", m.changefeedID.ID), - zap.Bool("skipped", skipped), - zap.String("error", err.Error())) + return false } -func (m *SinkManager) startSinkWorkers(ctx context.Context, eg *errgroup.Group, splitTxn bool, enableOldValue bool) { +func (m *SinkManager) startSinkWorkers(ctx context.Context, eg *errgroup.Group, splitTxn bool) { for i := 0; i < sinkWorkerNum; i++ { w := newSinkWorker(m.changefeedID, m.sourceManager, m.sinkMemQuota, m.redoMemQuota, - m.eventCache, splitTxn, enableOldValue) + m.eventCache, splitTxn) m.sinkWorkers = append(m.sinkWorkers, w) eg.Go(func() error { return w.handleTasks(ctx, m.sinkTaskChan) }) } } -func (m *SinkManager) startRedoWorkers(ctx context.Context, eg *errgroup.Group, enableOldValue bool) { +func (m *SinkManager) startRedoWorkers(ctx context.Context, eg *errgroup.Group) { for i := 0; i < redoWorkerNum; i++ { w := newRedoWorker(m.changefeedID, m.sourceManager, m.redoMemQuota, - m.redoDMLMgr, m.eventCache, enableOldValue) + m.redoDMLMgr, m.eventCache) m.redoWorkers = append(m.redoWorkers, w) eg.Go(func() error { return w.handleTasks(ctx, m.redoTaskChan) }) } @@ -419,7 +420,7 @@ func (m *SinkManager) backgroundGC(errors chan<- error) { for { select { case <-m.managerCtx.Done(): - log.Info("Background GC is stoped because context is canceled", + log.Info("Background GC is stopped because context is canceled", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID)) return @@ -435,7 +436,7 @@ func (m *SinkManager) backgroundGC(errors chan<- error) { if time.Since(sink.lastCleanTime) < cleanTableInterval { return true } - checkpointTs, _, _ := sink.getCheckpointTs() + checkpointTs := sink.getCheckpointTs() resolvedMark := checkpointTs.ResolvedMark() if resolvedMark == 0 { return true @@ -767,30 +768,13 @@ func (m *SinkManager) UpdateReceivedSorterResolvedTs(span tablepb.Span, ts model } // UpdateBarrierTs update all tableSink's barrierTs in the SinkManager -func (m *SinkManager) UpdateBarrierTs( - globalBarrierTs model.Ts, - tableBarrier map[model.TableID]model.Ts, -) { +func (m *SinkManager) UpdateBarrierTs(globalBarrierTs model.Ts, tableBarrier map[model.TableID]model.Ts) { m.tableSinks.Range(func(span tablepb.Span, value interface{}) bool { - tableSink := value.(*tableSinkWrapper) - lastBarrierTs := tableSink.barrierTs.Load() - // It is safe to do not use compare and swap here. - // Only the processor will update the barrier ts. - // Other goroutines will only read the barrier ts. - // So it is safe to do not use compare and swap here, just Load and Store. - if tableBarrierTs, ok := tableBarrier[tableSink.span.TableID]; ok { - barrierTs := tableBarrierTs - if barrierTs > globalBarrierTs { - barrierTs = globalBarrierTs - } - if barrierTs > lastBarrierTs { - tableSink.barrierTs.Store(barrierTs) - } - } else { - if globalBarrierTs > lastBarrierTs { - tableSink.barrierTs.Store(globalBarrierTs) - } + barrierTs := globalBarrierTs + if tableBarrierTs, ok := tableBarrier[span.TableID]; ok && tableBarrierTs < globalBarrierTs { + barrierTs = tableBarrierTs } + value.(*tableSinkWrapper).updateBarrierTs(barrierTs) return true }) } @@ -907,7 +891,7 @@ func (m *SinkManager) RemoveTable(span tablepb.Span) { zap.String("changefeed", m.changefeedID.ID), zap.Stringer("span", &span)) } - checkpointTs, _, _ := value.(*tableSinkWrapper).getCheckpointTs() + checkpointTs := value.(*tableSinkWrapper).getCheckpointTs() log.Info("Remove table sink successfully", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), @@ -976,27 +960,27 @@ func (m *SinkManager) GetTableStats(span tablepb.Span) TableStats { } tableSink := value.(*tableSinkWrapper) - checkpointTs, version, advanced := tableSink.getCheckpointTs() + checkpointTs := tableSink.getCheckpointTs() m.sinkMemQuota.Release(span, checkpointTs) m.redoMemQuota.Release(span, checkpointTs) advanceTimeoutInSec := util.GetOrZero(m.changefeedInfo.Config.Sink.AdvanceTimeoutInSec) if advanceTimeoutInSec <= 0 { - log.Warn("AdvanceTimeoutInSec is not set, use default value", zap.Any("sinkConfig", m.changefeedInfo.Config.Sink)) advanceTimeoutInSec = config.DefaultAdvanceTimeoutInSec } stuckCheck := time.Duration(advanceTimeoutInSec) * time.Second - if version > 0 && time.Since(advanced) > stuckCheck && - oracle.GetTimeFromTS(tableSink.getUpperBoundTs()).Sub(oracle.GetTimeFromTS(checkpointTs.Ts)) > stuckCheck { - log.Warn("Table checkpoint is stuck too long, will restart the sink backend", - zap.String("namespace", m.changefeedID.Namespace), - zap.String("changefeed", m.changefeedID.ID), - zap.Stringer("span", &span), - zap.Any("checkpointTs", checkpointTs), - zap.Float64("stuckCheck", stuckCheck.Seconds()), - zap.Uint64("factoryVersion", version)) - tableSink.updateTableSinkAdvanced() - m.putSinkFactoryError(errors.New("table sink stuck"), version) + + if m.needsStuckCheck() { + isStuck, sinkVersion := tableSink.sinkMaybeStuck(stuckCheck) + if isStuck && m.putSinkFactoryError(errors.New("table sink stuck"), sinkVersion) { + log.Warn("Table checkpoint is stuck too long, will restart the sink backend", + zap.String("namespace", m.changefeedID.Namespace), + zap.String("changefeed", m.changefeedID.ID), + zap.Stringer("span", &span), + zap.Any("checkpointTs", checkpointTs), + zap.Float64("stuckCheck", stuckCheck.Seconds()), + zap.Uint64("factoryVersion", sinkVersion)) + } } var resolvedTs model.Ts @@ -1007,14 +991,14 @@ func (m *SinkManager) GetTableStats(span tablepb.Span) TableStats { resolvedTs = tableSink.getReceivedSorterResolvedTs() } - if resolvedTs < checkpointTs.ResolvedMark() { - log.Error("sinkManager: resolved ts should not less than checkpoint ts", + sinkUpperBound := tableSink.getUpperBoundTs() + if sinkUpperBound < checkpointTs.ResolvedMark() { + log.Panic("sinkManager: sink upperbound should not less than checkpoint ts", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), zap.Stringer("span", &span), - zap.Uint64("resolvedTs", resolvedTs), - zap.Any("checkpointTs", checkpointTs), - zap.Uint64("barrierTs", tableSink.barrierTs.Load())) + zap.Uint64("upperbound", sinkUpperBound), + zap.Any("checkpointTs", checkpointTs)) } return TableStats{ CheckpointTs: checkpointTs.ResolvedMark(), diff --git a/cdc/processor/sinkmanager/manager_test.go b/cdc/processor/sinkmanager/manager_test.go index d79657e749f..b70e17fc3ab 100644 --- a/cdc/processor/sinkmanager/manager_test.go +++ b/cdc/processor/sinkmanager/manager_test.go @@ -197,7 +197,7 @@ func TestGenerateTableSinkTaskWithBarrierTs(t *testing.T) { require.Eventually(t, func() bool { tableSink, ok := manager.tableSinks.Load(span) require.True(t, ok) - checkpointTS, _, _ := tableSink.(*tableSinkWrapper).getCheckpointTs() + checkpointTS := tableSink.(*tableSinkWrapper).getCheckpointTs() return checkpointTS.ResolvedMark() == 4 }, 5*time.Second, 10*time.Millisecond) } @@ -228,7 +228,7 @@ func TestGenerateTableSinkTaskWithResolvedTs(t *testing.T) { require.Eventually(t, func() bool { tableSink, ok := manager.tableSinks.Load(span) require.True(t, ok) - checkpointTS, _, _ := tableSink.(*tableSinkWrapper).getCheckpointTs() + checkpointTS := tableSink.(*tableSinkWrapper).getCheckpointTs() return checkpointTS.ResolvedMark() == 3 }, 5*time.Second, 10*time.Millisecond) } @@ -283,7 +283,7 @@ func TestDoNotGenerateTableSinkTaskWhenTableIsNotReplicating(t *testing.T) { tableSink, ok := manager.tableSinks.Load(span) require.True(t, ok) require.NotNil(t, tableSink) - checkpointTS, _, _ := tableSink.(*tableSinkWrapper).getCheckpointTs() + checkpointTS := tableSink.(*tableSinkWrapper).getCheckpointTs() require.Equal(t, uint64(1), checkpointTS.Ts) } @@ -357,3 +357,18 @@ func TestSinkManagerRunWithErrors(t *testing.T) { log.Panic("must get an error instead of a timeout") } } + +func TestSinkManagerNeedsStuckCheck(t *testing.T) { + t.Parallel() + + ctx, cancel := context.WithCancel(context.Background()) + errCh := make(chan error, 16) + changefeedInfo := getChangefeedInfo() + manager, _, _ := CreateManagerWithMemEngine(t, ctx, model.DefaultChangeFeedID("1"), changefeedInfo, errCh) + defer func() { + cancel() + manager.Close() + }() + + require.False(t, manager.needsStuckCheck()) +} diff --git a/cdc/processor/sinkmanager/redo_log_worker.go b/cdc/processor/sinkmanager/redo_log_worker.go index 20f5bd18ea9..89f2c7d604d 100644 --- a/cdc/processor/sinkmanager/redo_log_worker.go +++ b/cdc/processor/sinkmanager/redo_log_worker.go @@ -33,7 +33,6 @@ type redoWorker struct { memQuota *memquota.MemQuota redoDMLManager redo.DMLManager eventCache *redoEventCache - enableOldValue bool } func newRedoWorker( @@ -42,7 +41,6 @@ func newRedoWorker( quota *memquota.MemQuota, redoDMLMgr redo.DMLManager, eventCache *redoEventCache, - enableOldValue bool, ) *redoWorker { return &redoWorker{ changefeedID: changefeedID, @@ -50,7 +48,6 @@ func newRedoWorker( memQuota: quota, redoDMLManager: redoDMLMgr, eventCache: eventCache, - enableOldValue: enableOldValue, } } @@ -68,22 +65,23 @@ func (w *redoWorker) handleTasks(ctx context.Context, taskChan <-chan *redoTask) } func (w *redoWorker) handleTask(ctx context.Context, task *redoTask) (finalErr error) { + advancer := newRedoLogAdvancer(task, w.memQuota, requestMemSize, w.redoDMLManager) + // The task is finished and some required memory isn't used. + defer advancer.cleanup() + lowerBound, upperBound := validateAndAdjustBound( w.changefeedID, &task.span, task.lowerBound, task.getUpperBound(task.tableSink.getReceivedSorterResolvedTs()), ) + advancer.lastPos = lowerBound.Prev() var cache *eventAppender if w.eventCache != nil { cache = w.eventCache.maybeCreateAppender(task.span, lowerBound) } - advancer := newRedoLogAdvancer(task, w.memQuota, requestMemSize, w.redoDMLManager) - // The task is finished and some required memory isn't used. - defer advancer.cleanup() - iter := w.sourceManager.FetchByTable(task.span, lowerBound, upperBound, w.memQuota) allEventCount := 0 cachedSize := uint64(0) @@ -127,11 +125,7 @@ func (w *redoWorker) handleTask(ctx context.Context, task *redoTask) (finalErr e cache.pushBatch(nil, 0, upperBound) } - return advancer.finish( - ctx, - cachedSize, - upperBound, - ) + return advancer.finish(ctx, cachedSize, upperBound) } allEventCount += 1 @@ -148,10 +142,7 @@ func (w *redoWorker) handleTask(ctx context.Context, task *redoTask) (finalErr e if e.Row != nil { // For all events, we add table replicate ts, so mysql sink can determine safe-mode. e.Row.ReplicatingTs = task.tableSink.replicateTs - x, size, err = convertRowChangedEvents(w.changefeedID, task.span, w.enableOldValue, e) - if err != nil { - return errors.Trace(err) - } + x, size = handleRowChangedEvents(w.changefeedID, task.span, e) advancer.appendEvents(x, size) } diff --git a/cdc/processor/sinkmanager/redo_log_worker_test.go b/cdc/processor/sinkmanager/redo_log_worker_test.go index 038d366eaf1..98e107d067d 100644 --- a/cdc/processor/sinkmanager/redo_log_worker_test.go +++ b/cdc/processor/sinkmanager/redo_log_worker_test.go @@ -75,7 +75,7 @@ func (suite *redoLogWorkerSuite) createWorker( eventCache := newRedoEventCache(suite.testChangefeedID, 1024) return newRedoWorker(suite.testChangefeedID, sm, quota, - redoDMLManager, eventCache, false), sortEngine, redoDMLManager + redoDMLManager, eventCache), sortEngine, redoDMLManager } func (suite *redoLogWorkerSuite) addEventsToSortEngine( @@ -284,3 +284,47 @@ func (suite *redoLogWorkerSuite) TestHandleTaskWithSplitTxnAndAdvanceIfNoWorkloa cancel() wg.Wait() } + +// When starts to handle a task, advancer.lastPos should be set to a correct position. +// Otherwise if advancer.lastPos isn't updated during scanning, callback will get an +// invalid `advancer.lastPos`. +func (suite *redoLogWorkerSuite) TestHandleTaskWithoutMemory() { + ctx, cancel := context.WithCancel(context.Background()) + events := []*model.PolymorphicEvent{ + genPolymorphicEvent(1, 3, suite.testSpan), + genPolymorphicResolvedEvent(4), + } + w, e, _ := suite.createWorker(ctx, 0) + defer w.memQuota.Close() + suite.addEventsToSortEngine(events, e) + + taskChan := make(chan *redoTask) + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + err := w.handleTasks(ctx, taskChan) + require.Equal(suite.T(), context.Canceled, err) + }() + + wrapper, sink := createTableSinkWrapper(suite.testChangefeedID, suite.testSpan) + defer sink.Close() + + chShouldBeClosed := make(chan struct{}, 1) + callback := func(lastWritePos engine.Position) { + require.Equal(suite.T(), genLowerBound().Prev(), lastWritePos) + close(chShouldBeClosed) + } + taskChan <- &redoTask{ + span: suite.testSpan, + lowerBound: genLowerBound(), + getUpperBound: genUpperBoundGetter(4), + tableSink: wrapper, + callback: callback, + isCanceled: func() bool { return true }, + } + + <-chShouldBeClosed + cancel() + wg.Wait() +} diff --git a/cdc/processor/sinkmanager/table_sink_advancer_test.go b/cdc/processor/sinkmanager/table_sink_advancer_test.go index b1c2af8303a..f28c5f9496c 100644 --- a/cdc/processor/sinkmanager/table_sink_advancer_test.go +++ b/cdc/processor/sinkmanager/table_sink_advancer_test.go @@ -136,7 +136,7 @@ func (suite *tableSinkAdvancerSuite) TestAdvanceTableSinkWithBatchID() { expectedResolvedTs := model.NewResolvedTs(2) expectedResolvedTs.Mode = model.BatchResolvedMode expectedResolvedTs.BatchID = 1 - checkpointTs, _, _ := task.tableSink.getCheckpointTs() + checkpointTs := task.tableSink.getCheckpointTs() require.Equal(suite.T(), expectedResolvedTs, checkpointTs) } @@ -151,7 +151,7 @@ func (suite *tableSinkAdvancerSuite) TestAdvanceTableSink() { require.NoError(suite.T(), err) expectedResolvedTs := model.NewResolvedTs(2) - checkpointTs, _, _ := task.tableSink.getCheckpointTs() + checkpointTs := task.tableSink.getCheckpointTs() require.Equal(suite.T(), expectedResolvedTs, checkpointTs) } @@ -290,7 +290,7 @@ func (suite *tableSinkAdvancerSuite) TestAdvanceTheSameCommitTsEventsWithCommitF require.Len(suite.T(), sink.GetEvents(), 3) sink.AckAllEvents() require.Eventually(suite.T(), func() bool { - checkpointTs, _, _ := task.tableSink.getCheckpointTs() + checkpointTs := task.tableSink.getCheckpointTs() return checkpointTs == model.NewResolvedTs(2) }, 5*time.Second, 10*time.Millisecond) require.Equal(suite.T(), uint64(0), advancer.committedTxnSize) @@ -337,7 +337,7 @@ func (suite *tableSinkAdvancerSuite) TestAdvanceTheSameCommitTsEventsWithoutComm expectedResolvedTs := model.NewResolvedTs(3) expectedResolvedTs.Mode = model.BatchResolvedMode expectedResolvedTs.BatchID = 1 - checkpointTs, _, _ := task.tableSink.getCheckpointTs() + checkpointTs := task.tableSink.getCheckpointTs() return checkpointTs == expectedResolvedTs }, 5*time.Second, 10*time.Millisecond) require.Equal(suite.T(), uint64(0), advancer.committedTxnSize) @@ -388,7 +388,7 @@ func (suite *tableSinkAdvancerSuite) TestAdvanceDifferentCommitTsEventsWithSplit expectedResolvedTs := model.NewResolvedTs(3) expectedResolvedTs.Mode = model.BatchResolvedMode expectedResolvedTs.BatchID = 1 - checkpointTs, _, _ := task.tableSink.getCheckpointTs() + checkpointTs := task.tableSink.getCheckpointTs() return checkpointTs == expectedResolvedTs }, 5*time.Second, 10*time.Millisecond) require.Equal(suite.T(), uint64(0), advancer.committedTxnSize) @@ -443,7 +443,7 @@ func (suite *tableSinkAdvancerSuite) TestAdvanceDifferentCommitTsEventsWithoutSp sink.AckAllEvents() require.Eventually(suite.T(), func() bool { expectedResolvedTs := model.NewResolvedTs(2) - checkpointTs, _, _ := task.tableSink.getCheckpointTs() + checkpointTs := task.tableSink.getCheckpointTs() return checkpointTs == expectedResolvedTs }, 5*time.Second, 10*time.Millisecond) require.Equal(suite.T(), uint64(0), advancer.committedTxnSize) @@ -499,7 +499,7 @@ func (suite *tableSinkAdvancerSuite) TestLastTimeAdvanceDifferentCommitTsEventsW sink.AckAllEvents() require.Eventually(suite.T(), func() bool { expectedResolvedTs := model.NewResolvedTs(2) - checkpointTs, _, _ := task.tableSink.getCheckpointTs() + checkpointTs := task.tableSink.getCheckpointTs() return checkpointTs == expectedResolvedTs }, 5*time.Second, 10*time.Millisecond) require.Equal(suite.T(), uint64(0), advancer.committedTxnSize) @@ -557,7 +557,7 @@ func (suite *tableSinkAdvancerSuite) TestTryAdvanceWhenExceedAvailableMem() { sink.AckAllEvents() require.Eventually(suite.T(), func() bool { expectedResolvedTs := model.NewResolvedTs(3) - checkpointTs, _, _ := task.tableSink.getCheckpointTs() + checkpointTs := task.tableSink.getCheckpointTs() return checkpointTs == expectedResolvedTs }, 5*time.Second, 10*time.Millisecond) require.Equal(suite.T(), uint64(0), advancer.committedTxnSize) @@ -607,7 +607,7 @@ func (suite *tableSinkAdvancerSuite) TestTryAdvanceWhenReachTheMaxUpdateIntSizeA sink.AckAllEvents() require.Eventually(suite.T(), func() bool { expectedResolvedTs := model.NewResolvedTs(3) - checkpointTs, _, _ := task.tableSink.getCheckpointTs() + checkpointTs := task.tableSink.getCheckpointTs() return checkpointTs == expectedResolvedTs }, 5*time.Second, 10*time.Millisecond) require.Equal(suite.T(), uint64(0), advancer.committedTxnSize) @@ -660,7 +660,7 @@ func (suite *tableSinkAdvancerSuite) TestFinish() { sink.AckAllEvents() require.Eventually(suite.T(), func() bool { expectedResolvedTs := model.NewResolvedTs(4) - checkpointTs, _, _ := task.tableSink.getCheckpointTs() + checkpointTs := task.tableSink.getCheckpointTs() return checkpointTs == expectedResolvedTs }, 5*time.Second, 10*time.Millisecond) require.Equal(suite.T(), uint64(0), advancer.committedTxnSize) @@ -710,7 +710,7 @@ func (suite *tableSinkAdvancerSuite) TestTryAdvanceAndForceAcquireWithoutSplitTx sink.AckAllEvents() require.Eventually(suite.T(), func() bool { expectedResolvedTs := model.NewResolvedTs(3) - checkpointTs, _, _ := task.tableSink.getCheckpointTs() + checkpointTs := task.tableSink.getCheckpointTs() return checkpointTs == expectedResolvedTs }, 5*time.Second, 10*time.Millisecond) require.Equal(suite.T(), uint64(0), advancer.committedTxnSize) @@ -775,7 +775,7 @@ func (suite *tableSinkAdvancerSuite) TestTryAdvanceAndBlockAcquireWithSplitTxn() <-down require.Eventually(suite.T(), func() bool { expectedResolvedTs := model.NewResolvedTs(3) - checkpointTs, _, _ := task.tableSink.getCheckpointTs() + checkpointTs := task.tableSink.getCheckpointTs() return checkpointTs == expectedResolvedTs }, 5*time.Second, 10*time.Millisecond) require.Equal(suite.T(), uint64(0), advancer.committedTxnSize) diff --git a/cdc/processor/sinkmanager/table_sink_worker.go b/cdc/processor/sinkmanager/table_sink_worker.go index 5c3033a33fe..ac77e424a2c 100644 --- a/cdc/processor/sinkmanager/table_sink_worker.go +++ b/cdc/processor/sinkmanager/table_sink_worker.go @@ -65,9 +65,6 @@ type sinkWorker struct { eventCache *redoEventCache // splitTxn indicates whether to split the transaction into multiple batches. splitTxn bool - // enableOldValue indicates whether to enable the old value feature. - // If it is enabled, we need to deal with the compatibility of the data format. - enableOldValue bool // Metrics. metricRedoEventCacheHit prometheus.Counter @@ -83,16 +80,14 @@ func newSinkWorker( redoQuota *memquota.MemQuota, eventCache *redoEventCache, splitTxn bool, - enableOldValue bool, ) *sinkWorker { return &sinkWorker{ - changefeedID: changefeedID, - sourceManager: sourceManager, - sinkMemQuota: sinkQuota, - redoMemQuota: redoQuota, - eventCache: eventCache, - splitTxn: splitTxn, - enableOldValue: enableOldValue, + changefeedID: changefeedID, + sourceManager: sourceManager, + sinkMemQuota: sinkQuota, + redoMemQuota: redoQuota, + eventCache: eventCache, + splitTxn: splitTxn, metricRedoEventCacheHit: RedoEventCacheAccess.WithLabelValues(changefeedID.Namespace, changefeedID.ID, "hit"), metricRedoEventCacheMiss: RedoEventCacheAccess.WithLabelValues(changefeedID.Namespace, changefeedID.ID, "miss"), @@ -129,23 +124,18 @@ func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr e &task.span, task.lowerBound, task.getUpperBound(task.tableSink.getUpperBoundTs())) - if w.eventCache != nil { - drained, err := w.fetchFromCache(task, &lowerBound, &upperBound) - if err != nil { - return errors.Trace(err) - } - // We have drained all events from the cache, we can return directly. - // No need to get events from the source manager again. - if drained { - task.callback(lowerBound.Prev()) - return nil - } - } + advancer.lastPos = lowerBound.Prev() allEventSize := uint64(0) allEventCount := 0 - // lowerBound and upperBound are both closed intervals. - iter := w.sourceManager.FetchByTable(task.span, lowerBound, upperBound, w.sinkMemQuota) + + callbackIsPerformed := false + performCallback := func(pos engine.Position) { + if !callbackIsPerformed { + task.callback(pos) + callbackIsPerformed = true + } + } defer func() { // Collect metrics. @@ -158,13 +148,6 @@ func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr e task.tableSink.updateRangeEventCounts(eventCount) } - if err := iter.Close(); err != nil { - log.Error("Sink worker fails to close iterator", - zap.String("namespace", w.changefeedID.Namespace), - zap.String("changefeed", w.changefeedID.ID), - zap.Stringer("span", &task.span), - zap.Error(err)) - } log.Debug("Sink task finished", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), @@ -179,7 +162,7 @@ func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr e // Otherwise we can't ensure all events before `lastPos` are emitted. if finalErr == nil { - task.callback(advancer.lastPos) + performCallback(advancer.lastPos) } else { switch errors.Cause(finalErr).(type) { // If it's a warning, close the table sink and wait all pending @@ -192,22 +175,53 @@ func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr e w.sinkMemQuota.ClearTable(task.tableSink.span) // Restart the table sink based on the checkpoint position. - if finalErr = task.tableSink.restart(ctx); finalErr == nil { - checkpointTs, _, _ := task.tableSink.getCheckpointTs() + if err := task.tableSink.restart(ctx); err == nil { + checkpointTs := task.tableSink.getCheckpointTs() ckpt := checkpointTs.ResolvedMark() lastWrittenPos := engine.Position{StartTs: ckpt - 1, CommitTs: ckpt} - task.callback(lastWrittenPos) + performCallback(lastWrittenPos) log.Info("table sink has been restarted", zap.String("namespace", w.changefeedID.Namespace), zap.String("changefeed", w.changefeedID.ID), zap.Stringer("span", &task.span), - zap.Any("lastWrittenPos", lastWrittenPos)) + zap.Any("lastWrittenPos", lastWrittenPos), + zap.String("sinkError", finalErr.Error())) + finalErr = err } default: } } }() + if w.eventCache != nil { + drained, err := w.fetchFromCache(task, &lowerBound, &upperBound) + failpoint.Inject("TableSinkWorkerFetchFromCache", func() { + err = tablesink.NewSinkInternalError(errors.New("TableSinkWorkerFetchFromCacheInjected")) + }) + if err != nil { + return errors.Trace(err) + } + if drained { + // If drained is true it means we have drained all events from the cache, + // we can return directly instead of get events from the source manager again. + performCallback(lowerBound.Prev()) + return nil + } + advancer.lastPos = lowerBound.Prev() + } + + // lowerBound and upperBound are both closed intervals. + iter := w.sourceManager.FetchByTable(task.span, lowerBound, upperBound, w.sinkMemQuota) + defer func() { + if err := iter.Close(); err != nil { + log.Error("Sink worker fails to close iterator", + zap.String("namespace", w.changefeedID.Namespace), + zap.String("changefeed", w.changefeedID.ID), + zap.Stringer("span", &task.span), + zap.Error(err)) + } + }() + // 1. We have enough memory to collect events. // 2. The task is not canceled. for advancer.hasEnoughMem() && !task.isCanceled() { @@ -236,11 +250,7 @@ func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr e if e.Row != nil { // For all rows, we add table replicate ts, so mysql sink can determine safe-mode. e.Row.ReplicatingTs = task.tableSink.replicateTs - x, size, err := convertRowChangedEvents(w.changefeedID, task.span, w.enableOldValue, e) - if err != nil { - return err - } - + x, size := handleRowChangedEvents(w.changefeedID, task.span, e) advancer.appendEvents(x, size) allEventSize += size } diff --git a/cdc/processor/sinkmanager/table_sink_worker_test.go b/cdc/processor/sinkmanager/table_sink_worker_test.go index 0eb9c7f7346..6e65c5e97b0 100644 --- a/cdc/processor/sinkmanager/table_sink_worker_test.go +++ b/cdc/processor/sinkmanager/table_sink_worker_test.go @@ -19,6 +19,7 @@ import ( "testing" "time" + "github.com/pingcap/failpoint" "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/processor/memquota" @@ -138,7 +139,7 @@ func (suite *tableSinkWorkerSuite) createWorker( quota.ForceAcquire(testEventSize) quota.AddTable(suite.testSpan) - return newSinkWorker(suite.testChangefeedID, sm, quota, nil, nil, splitTxn, false), sortEngine + return newSinkWorker(suite.testChangefeedID, sm, quota, nil, nil, splitTxn), sortEngine } func (suite *tableSinkWorkerSuite) addEventsToSortEngine( @@ -534,7 +535,7 @@ func (suite *tableSinkWorkerSuite) TestHandleTaskWithSplitTxnAndAdvanceTableWhen receivedEvents := sink.GetEvents() receivedEvents[0].Callback() require.Len(suite.T(), sink.GetEvents(), 1, "No more events should be sent to sink") - checkpointTs, _, _ := wrapper.getCheckpointTs() + checkpointTs := wrapper.getCheckpointTs() require.Equal(suite.T(), uint64(4), checkpointTs.ResolvedMark()) } @@ -580,7 +581,7 @@ func (suite *tableSinkWorkerSuite) TestHandleTaskWithSplitTxnAndAdvanceTableIfNo isCanceled: func() bool { return false }, } require.Eventually(suite.T(), func() bool { - checkpointTs, _, _ := wrapper.getCheckpointTs() + checkpointTs := wrapper.getCheckpointTs() return checkpointTs.ResolvedMark() == 4 }, 5*time.Second, 10*time.Millisecond, "Directly advance resolved mark to 4") cancel() @@ -638,7 +639,7 @@ func (suite *tableSinkWorkerSuite) TestHandleTaskUseDifferentBatchIDEveryTime() require.Equal(suite.T(), uint64(3), batchID.Load()) sink.AckAllEvents() require.Eventually(suite.T(), func() bool { - checkpointTs, _, _ := wrapper.getCheckpointTs() + checkpointTs := wrapper.getCheckpointTs() return checkpointTs.ResolvedMark() == 2 }, 5*time.Second, 10*time.Millisecond) @@ -666,3 +667,97 @@ func (suite *tableSinkWorkerSuite) TestHandleTaskUseDifferentBatchIDEveryTime() require.Equal(suite.T(), uint64(5), batchID.Load(), "The batchID should be 5, "+ "because the first task has 3 events, the second task has 1 event") } + +func (suite *tableSinkWorkerSuite) TestFetchFromCacheWithFailure() { + ctx, cancel := context.WithCancel(context.Background()) + events := []*model.PolymorphicEvent{ + genPolymorphicEvent(1, 3, suite.testSpan), + genPolymorphicEvent(1, 3, suite.testSpan), + genPolymorphicEvent(1, 3, suite.testSpan), + genPolymorphicResolvedEvent(4), + } + // Only for three events. + eventSize := uint64(testEventSize * 3) + w, e := suite.createWorker(ctx, eventSize, true) + w.eventCache = newRedoEventCache(suite.testChangefeedID, 1024*1024) + defer w.sinkMemQuota.Close() + suite.addEventsToSortEngine(events, e) + + _ = failpoint.Enable("github.com/pingcap/tiflow/cdc/processor/sinkmanager/TableSinkWorkerFetchFromCache", "return") + defer func() { + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/processor/sinkmanager/TableSinkWorkerFetchFromCache") + }() + + taskChan := make(chan *sinkTask) + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + err := w.handleTasks(ctx, taskChan) + require.Equal(suite.T(), context.Canceled, err) + }() + + wrapper, sink := createTableSinkWrapper(suite.testChangefeedID, suite.testSpan) + defer sink.Close() + + chShouldBeClosed := make(chan struct{}, 1) + callback := func(lastWritePos engine.Position) { + close(chShouldBeClosed) + } + taskChan <- &sinkTask{ + span: suite.testSpan, + lowerBound: genLowerBound(), + getUpperBound: genUpperBoundGetter(4), + tableSink: wrapper, + callback: callback, + isCanceled: func() bool { return false }, + } + + <-chShouldBeClosed + cancel() + wg.Wait() +} + +// When starts to handle a task, advancer.lastPos should be set to a correct position. +// Otherwise if advancer.lastPos isn't updated during scanning, callback will get an +// invalid `advancer.lastPos`. +func (suite *tableSinkWorkerSuite) TestHandleTaskWithoutMemory() { + ctx, cancel := context.WithCancel(context.Background()) + events := []*model.PolymorphicEvent{ + genPolymorphicEvent(1, 3, suite.testSpan), + genPolymorphicResolvedEvent(4), + } + w, e := suite.createWorker(ctx, 0, true) + defer w.sinkMemQuota.Close() + suite.addEventsToSortEngine(events, e) + + taskChan := make(chan *sinkTask) + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + err := w.handleTasks(ctx, taskChan) + require.Equal(suite.T(), context.Canceled, err) + }() + + wrapper, sink := createTableSinkWrapper(suite.testChangefeedID, suite.testSpan) + defer sink.Close() + + chShouldBeClosed := make(chan struct{}, 1) + callback := func(lastWritePos engine.Position) { + require.Equal(suite.T(), genLowerBound().Prev(), lastWritePos) + close(chShouldBeClosed) + } + taskChan <- &sinkTask{ + span: suite.testSpan, + lowerBound: genLowerBound(), + getUpperBound: genUpperBoundGetter(4), + tableSink: wrapper, + callback: callback, + isCanceled: func() bool { return true }, + } + + <-chShouldBeClosed + cancel() + wg.Wait() +} diff --git a/cdc/processor/sinkmanager/table_sink_wrapper.go b/cdc/processor/sinkmanager/table_sink_wrapper.go index 1f6f7dca3c7..2495a3b115d 100644 --- a/cdc/processor/sinkmanager/table_sink_wrapper.go +++ b/cdc/processor/sinkmanager/table_sink_wrapper.go @@ -33,7 +33,7 @@ import ( "go.uber.org/zap" ) -var version uint64 = 0 +var tableSinkWrapperVersion uint64 = 0 // tableSinkWrapper is a wrapper of TableSink, it is used in SinkManager to manage TableSink. // Because in the SinkManager, we write data to TableSink and RedoManager concurrently, @@ -46,15 +46,18 @@ type tableSinkWrapper struct { // tableSpan used for logging. span tablepb.Span - tableSinkCreater func() (tablesink.TableSink, uint64) + tableSinkCreator func() (tablesink.TableSink, uint64) // tableSink is the underlying sink. tableSink struct { sync.RWMutex - s tablesink.TableSink - version uint64 // it's generated by `tableSinkCreater`. + s tablesink.TableSink + version uint64 // it's generated by `tableSinkCreater`. + + innerMu sync.Mutex + advanced time.Time + resolvedTs model.ResolvedTs checkpointTs model.ResolvedTs - advanced atomic.Int64 } // state used to control the lifecycle of the table. @@ -108,10 +111,10 @@ func newTableSinkWrapper( genReplicateTs func(ctx context.Context) (model.Ts, error), ) *tableSinkWrapper { res := &tableSinkWrapper{ - version: atomic.AddUint64(&version, 1), + version: atomic.AddUint64(&tableSinkWrapperVersion, 1), changefeed: changefeed, span: span, - tableSinkCreater: tableSinkCreater, + tableSinkCreator: tableSinkCreater, state: &state, startTs: startTs, targetTs: targetTs, @@ -120,7 +123,8 @@ func newTableSinkWrapper( res.tableSink.version = 0 res.tableSink.checkpointTs = model.NewResolvedTs(startTs) - res.updateTableSinkAdvanced() + res.tableSink.resolvedTs = model.NewResolvedTs(startTs) + res.tableSink.advanced = time.Now() res.receivedSorterResolvedTs.Store(startTs) res.barrierTs.Store(startTs) @@ -175,6 +179,15 @@ func (t *tableSinkWrapper) appendRowChangedEvents(events ...*model.RowChangedEve return nil } +func (t *tableSinkWrapper) updateBarrierTs(ts model.Ts) { + for { + old := t.barrierTs.Load() + if ts <= old || t.barrierTs.CompareAndSwap(old, ts) { + break + } + } +} + func (t *tableSinkWrapper) updateReceivedSorterResolvedTs(ts model.Ts) { for { old := t.receivedSorterResolvedTs.Load() @@ -197,33 +210,28 @@ func (t *tableSinkWrapper) updateResolvedTs(ts model.ResolvedTs) error { // If it's nil it means it's closed. return tablesink.NewSinkInternalError(errors.New("table sink cleared")) } + t.tableSink.innerMu.Lock() + defer t.tableSink.innerMu.Unlock() + t.tableSink.resolvedTs = ts return t.tableSink.s.UpdateResolvedTs(ts) } -// getCheckpointTs returns -// 1. checkpoint timestamp of the table; -// 2. the table sink version, which comes from `tableSinkCreater`; -// 3. recent time of the table is advanced. -func (t *tableSinkWrapper) getCheckpointTs() (model.ResolvedTs, uint64, time.Time) { +func (t *tableSinkWrapper) getCheckpointTs() model.ResolvedTs { t.tableSink.RLock() defer t.tableSink.RUnlock() + t.tableSink.innerMu.Lock() + defer t.tableSink.innerMu.Unlock() + if t.tableSink.s != nil { checkpointTs := t.tableSink.s.GetCheckpointTs() if t.tableSink.checkpointTs.Less(checkpointTs) { t.tableSink.checkpointTs = checkpointTs - t.updateTableSinkAdvanced() + t.tableSink.advanced = time.Now() + } else if !checkpointTs.Less(t.tableSink.resolvedTs) { + t.tableSink.advanced = time.Now() } } - advanced := time.Unix(t.tableSink.advanced.Load(), 0) - return t.tableSink.checkpointTs, t.tableSink.version, advanced -} - -func (t *tableSinkWrapper) updateTableSinkAdvanced() { - curr := t.tableSink.advanced.Load() - now := time.Now().Unix() - if now > curr { - t.tableSink.advanced.CompareAndSwap(curr, now) - } + return t.tableSink.checkpointTs } func (t *tableSinkWrapper) getReceivedSorterResolvedTs() model.Ts { @@ -294,9 +302,9 @@ func (t *tableSinkWrapper) initTableSink() bool { t.tableSink.Lock() defer t.tableSink.Unlock() if t.tableSink.s == nil { - t.tableSink.s, t.tableSink.version = t.tableSinkCreater() + t.tableSink.s, t.tableSink.version = t.tableSinkCreator() if t.tableSink.s != nil { - t.updateTableSinkAdvanced() + t.tableSink.advanced = time.Now() return true } return false @@ -342,12 +350,15 @@ func (t *tableSinkWrapper) doTableSinkClear() { return } checkpointTs := t.tableSink.s.GetCheckpointTs() + t.tableSink.innerMu.Lock() if t.tableSink.checkpointTs.Less(checkpointTs) { t.tableSink.checkpointTs = checkpointTs } + t.tableSink.resolvedTs = checkpointTs + t.tableSink.advanced = time.Now() + t.tableSink.innerMu.Unlock() t.tableSink.s = nil t.tableSink.version = 0 - t.tableSink.advanced.Store(time.Now().Unix()) } // When the attached sink fail, there can be some events that have already been @@ -417,12 +428,29 @@ func (t *tableSinkWrapper) cleanRangeEventCounts(upperBound engine.Position, min return shouldClean } -// convertRowChangedEvents uses to convert RowChangedEvents to TableSinkRowChangedEvents. -// It will deal with the old value compatibility. -func convertRowChangedEvents( - changefeed model.ChangeFeedID, span tablepb.Span, enableOldValue bool, +func (t *tableSinkWrapper) sinkMaybeStuck(stuckCheck time.Duration) (bool, uint64) { + t.getCheckpointTs() + + t.tableSink.RLock() + defer t.tableSink.RUnlock() + t.tableSink.innerMu.Lock() + defer t.tableSink.innerMu.Unlock() + + // What these conditions mean: + // 1. the table sink has been associated with a valid sink; + // 2. its checkpoint hasn't been advanced for a while; + version := t.tableSink.version + advanced := t.tableSink.advanced + if version > 0 && time.Since(advanced) > stuckCheck { + return true, version + } + return false, uint64(0) +} + +func handleRowChangedEvents( + changefeed model.ChangeFeedID, span tablepb.Span, events ...*model.PolymorphicEvent, -) ([]*model.RowChangedEvent, uint64, error) { +) ([]*model.RowChangedEvent, uint64) { size := 0 rowChangedEvents := make([]*model.RowChangedEvent, 0, len(events)) for _, e := range events { @@ -435,12 +463,11 @@ func convertRowChangedEvents( continue } - colLen := len(e.Row.Columns) - preColLen := len(e.Row.PreColumns) + rowEvent := e.Row // Some transactions could generate empty row change event, such as // begin; insert into t (id) values (1); delete from t where id=1; commit; // Just ignore these row changed events. - if colLen == 0 && preColLen == 0 { + if len(rowEvent.Columns) == 0 && len(rowEvent.PreColumns) == 0 { log.Warn("skip emit empty row event", zap.Stringer("span", &span), zap.String("namespace", changefeed.Namespace), @@ -449,86 +476,10 @@ func convertRowChangedEvents( continue } - size += e.Row.ApproximateBytes() - - // This indicates that it is an update event, - // and after enable old value internally by default(but disable in the configuration). - // We need to handle the update event to be compatible with the old format. - if e.Row.IsUpdate() && !enableOldValue { - if shouldSplitUpdateEvent(e) { - deleteEvent, insertEvent, err := splitUpdateEvent(e) - if err != nil { - return nil, 0, errors.Trace(err) - } - // NOTICE: Please do not change the order, the delete event always comes before the insert event. - rowChangedEvents = append(rowChangedEvents, deleteEvent.Row, insertEvent.Row) - } else { - // If the handle key columns are not updated, PreColumns is directly ignored. - e.Row.PreColumns = nil - rowChangedEvents = append(rowChangedEvents, e.Row) - } - } else { - rowChangedEvents = append(rowChangedEvents, e.Row) - } - } - return rowChangedEvents, uint64(size), nil -} - -// shouldSplitUpdateEvent determines if the split event is needed to align the old format based on -// whether the handle key column has been modified. -// If the handle key column is modified, -// we need to use splitUpdateEvent to split the update event into a delete and an insert event. -func shouldSplitUpdateEvent(updateEvent *model.PolymorphicEvent) bool { - // nil event will never be split. - if updateEvent == nil { - return false - } - - for i := range updateEvent.Row.Columns { - col := updateEvent.Row.Columns[i] - preCol := updateEvent.Row.PreColumns[i] - if col != nil && col.Flag.IsHandleKey() && preCol != nil && preCol.Flag.IsHandleKey() { - colValueString := model.ColumnValueString(col.Value) - preColValueString := model.ColumnValueString(preCol.Value) - // If one handle key columns is updated, we need to split the event row. - if colValueString != preColValueString { - return true - } - } - } - return false -} - -// splitUpdateEvent splits an update event into a delete and an insert event. -func splitUpdateEvent( - updateEvent *model.PolymorphicEvent, -) (*model.PolymorphicEvent, *model.PolymorphicEvent, error) { - if updateEvent == nil { - return nil, nil, errors.New("nil event cannot be split") + size += rowEvent.ApproximateBytes() + rowChangedEvents = append(rowChangedEvents, rowEvent) } - - // If there is an update to handle key columns, - // we need to split the event into two events to be compatible with the old format. - // NOTICE: Here we don't need a full deep copy because - // our two events need Columns and PreColumns respectively, - // so it won't have an impact and no more full deep copy wastes memory. - deleteEvent := *updateEvent - deleteEventRow := *updateEvent.Row - deleteEventRowKV := *updateEvent.RawKV - deleteEvent.Row = &deleteEventRow - deleteEvent.RawKV = &deleteEventRowKV - - deleteEvent.Row.Columns = nil - - insertEvent := *updateEvent - insertEventRow := *updateEvent.Row - insertEventRowKV := *updateEvent.RawKV - insertEvent.Row = &insertEventRow - insertEvent.RawKV = &insertEventRowKV - // NOTICE: clean up pre cols for insert event. - insertEvent.Row.PreColumns = nil - - return &deleteEvent, &insertEvent, nil + return rowChangedEvents, uint64(size) } func genReplicateTs(ctx context.Context, pdClient pd.Client) (model.Ts, error) { diff --git a/cdc/processor/sinkmanager/table_sink_wrapper_test.go b/cdc/processor/sinkmanager/table_sink_wrapper_test.go index c5dd31efdd9..91688e974a3 100644 --- a/cdc/processor/sinkmanager/table_sink_wrapper_test.go +++ b/cdc/processor/sinkmanager/table_sink_wrapper_test.go @@ -18,14 +18,17 @@ import ( "math" "sync" "testing" + "time" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/cdc/sink/dmlsink" "github.com/pingcap/tiflow/cdc/sink/tablesink" + "github.com/pingcap/tiflow/pkg/sink" "github.com/pingcap/tiflow/pkg/spanz" "github.com/prometheus/client_golang/prometheus" "github.com/stretchr/testify/require" + "github.com/tikv/client-go/v2/oracle" ) type mockSink struct { @@ -48,6 +51,10 @@ func (m *mockSink) WriteEvents(events ...*dmlsink.CallbackableEvent[*model.RowCh return nil } +func (m *mockSink) Scheme() string { + return sink.BlackHoleScheme +} + func (m *mockSink) GetEvents() []*dmlsink.CallbackableEvent[*model.RowChangedEvent] { m.mu.Lock() defer m.mu.Unlock() @@ -108,7 +115,7 @@ func createTableSinkWrapper( 100, func(_ context.Context) (model.Ts, error) { return math.MaxUint64, nil }, ) - wrapper.tableSink.s, wrapper.tableSink.version = wrapper.tableSinkCreater() + wrapper.tableSink.s, wrapper.tableSink.version = wrapper.tableSinkCreator() return wrapper, sink } @@ -145,26 +152,25 @@ func TestUpdateReceivedSorterResolvedTs(t *testing.T) { require.Equal(t, tablepb.TableStatePrepared, wrapper.getState()) } -func TestConvertNilRowChangedEvents(t *testing.T) { +func TestHandleNilRowChangedEvents(t *testing.T) { t.Parallel() events := []*model.PolymorphicEvent{nil} changefeedID := model.DefaultChangeFeedID("1") span := spanz.TableIDToComparableSpan(1) - enableOldVlaue := false - result, size, err := convertRowChangedEvents(changefeedID, span, enableOldVlaue, events...) - require.NoError(t, err) + result, size := handleRowChangedEvents(changefeedID, span, events...) require.Equal(t, 0, len(result)) require.Equal(t, uint64(0), size) } -func TestConvertEmptyRowChangedEvents(t *testing.T) { +func TestHandleEmptyRowChangedEvents(t *testing.T) { t.Parallel() events := []*model.PolymorphicEvent{ { StartTs: 1, CRTs: 2, + // the row had no columns Row: &model.RowChangedEvent{ StartTs: 1, CommitTs: 2, @@ -173,29 +179,17 @@ func TestConvertEmptyRowChangedEvents(t *testing.T) { } changefeedID := model.DefaultChangeFeedID("1") span := spanz.TableIDToComparableSpan(1) - enableOldValue := false - result, size, err := convertRowChangedEvents(changefeedID, span, enableOldValue, events...) - require.NoError(t, err) + + result, size := handleRowChangedEvents(changefeedID, span, events...) require.Equal(t, 0, len(result)) require.Equal(t, uint64(0), size) } -func TestConvertRowChangedEventsWhenEnableOldValue(t *testing.T) { +func TestHandleRowChangedEventNormalEvent(t *testing.T) { t.Parallel() + // Update non-unique key. columns := []*model.Column{ - { - Name: "col1", - Flag: model.BinaryFlag, - Value: "col1-value-updated", - }, - { - Name: "col2", - Flag: model.HandleKeyFlag, - Value: "col2-value-updated", - }, - } - preColumns := []*model.Column{ { Name: "col1", Flag: model.BinaryFlag, @@ -203,48 +197,7 @@ func TestConvertRowChangedEventsWhenEnableOldValue(t *testing.T) { }, { Name: "col2", - Flag: model.HandleKeyFlag, - Value: "col2-value", - }, - } - - events := []*model.PolymorphicEvent{ - { - CRTs: 1, - RawKV: &model.RawKVEntry{OpType: model.OpTypePut}, - Row: &model.RowChangedEvent{ - CommitTs: 1, - Columns: columns, - PreColumns: preColumns, - Table: &model.TableName{ - Schema: "test", - Table: "test", - }, - }, - }, - } - changefeedID := model.DefaultChangeFeedID("1") - span := spanz.TableIDToComparableSpan(1) - enableOldValue := true - result, size, err := convertRowChangedEvents(changefeedID, span, enableOldValue, events...) - require.NoError(t, err) - require.Equal(t, 1, len(result)) - require.Equal(t, uint64(224), size) -} - -func TestConvertRowChangedEventsWhenDisableOldValue(t *testing.T) { - t.Parallel() - - // Update handle key. - columns := []*model.Column{ - { - Name: "col1", - Flag: model.BinaryFlag, - Value: "col1-value-updated", - }, - { - Name: "col2", - Flag: model.HandleKeyFlag, + Flag: model.HandleKeyFlag | model.UniqueKeyFlag, Value: "col2-value-updated", }, } @@ -256,7 +209,7 @@ func TestConvertRowChangedEventsWhenDisableOldValue(t *testing.T) { }, { Name: "col2", - Flag: model.HandleKeyFlag, + Flag: model.HandleKeyFlag | model.UniqueKeyFlag, Value: "col2-value", }, } @@ -278,55 +231,7 @@ func TestConvertRowChangedEventsWhenDisableOldValue(t *testing.T) { } changefeedID := model.DefaultChangeFeedID("1") span := spanz.TableIDToComparableSpan(1) - enableOldValue := false - result, size, err := convertRowChangedEvents(changefeedID, span, enableOldValue, events...) - require.NoError(t, err) - require.Equal(t, 2, len(result)) - require.Equal(t, uint64(224), size) - - // Update non-handle key. - columns = []*model.Column{ - { - Name: "col1", - Flag: model.BinaryFlag, - Value: "col1-value-updated", - }, - { - Name: "col2", - Flag: model.HandleKeyFlag, - Value: "col2-value", - }, - } - preColumns = []*model.Column{ - { - Name: "col1", - Flag: model.BinaryFlag, - Value: "col1-value", - }, - { - Name: "col2", - Flag: model.HandleKeyFlag, - Value: "col2-value", - }, - } - - events = []*model.PolymorphicEvent{ - { - CRTs: 1, - RawKV: &model.RawKVEntry{OpType: model.OpTypePut}, - Row: &model.RowChangedEvent{ - CommitTs: 1, - Columns: columns, - PreColumns: preColumns, - Table: &model.TableName{ - Schema: "test", - Table: "test", - }, - }, - }, - } - result, size, err = convertRowChangedEvents(changefeedID, span, enableOldValue, events...) - require.NoError(t, err) + result, size := handleRowChangedEvents(changefeedID, span, events...) require.Equal(t, 1, len(result)) require.Equal(t, uint64(224), size) } @@ -358,6 +263,110 @@ func TestNewTableSinkWrapper(t *testing.T) { require.NotNil(t, wrapper) require.Equal(t, uint64(10), wrapper.getUpperBoundTs()) require.Equal(t, uint64(10), wrapper.getReceivedSorterResolvedTs()) - checkpointTs, _, _ := wrapper.getCheckpointTs() + checkpointTs := wrapper.getCheckpointTs() require.Equal(t, uint64(10), checkpointTs.ResolvedMark()) } + +func TestTableSinkWrapperSinkVersion(t *testing.T) { + t.Parallel() + + innerTableSink := tablesink.New[*model.RowChangedEvent]( + model.ChangeFeedID{}, tablepb.Span{}, model.Ts(0), + newMockSink(), &dmlsink.RowChangeEventAppender{}, + prometheus.NewCounter(prometheus.CounterOpts{}), + ) + version := new(uint64) + + wrapper := newTableSinkWrapper( + model.DefaultChangeFeedID("1"), + spanz.TableIDToComparableSpan(1), + func() (tablesink.TableSink, uint64) { return nil, 0 }, + tablepb.TableStatePrepared, + model.Ts(10), + model.Ts(20), + func(_ context.Context) (model.Ts, error) { return math.MaxUint64, nil }, + ) + + require.False(t, wrapper.initTableSink()) + + wrapper.tableSinkCreator = func() (tablesink.TableSink, uint64) { + *version += 1 + return innerTableSink, *version + } + + require.True(t, wrapper.initTableSink()) + require.Equal(t, wrapper.tableSink.version, uint64(1)) + + require.True(t, wrapper.asyncCloseTableSink()) + + wrapper.doTableSinkClear() + require.Nil(t, wrapper.tableSink.s) + require.Equal(t, wrapper.tableSink.version, uint64(0)) + + require.True(t, wrapper.initTableSink()) + require.Equal(t, wrapper.tableSink.version, uint64(2)) + + wrapper.closeTableSink() + + wrapper.doTableSinkClear() + require.Nil(t, wrapper.tableSink.s) + require.Equal(t, wrapper.tableSink.version, uint64(0)) +} + +func TestTableSinkWrapperSinkInner(t *testing.T) { + t.Parallel() + + innerTableSink := tablesink.New[*model.RowChangedEvent]( + model.ChangeFeedID{}, tablepb.Span{}, model.Ts(0), + newMockSink(), &dmlsink.RowChangeEventAppender{}, + prometheus.NewCounter(prometheus.CounterOpts{}), + ) + version := new(uint64) + + wrapper := newTableSinkWrapper( + model.DefaultChangeFeedID("1"), + spanz.TableIDToComparableSpan(1), + func() (tablesink.TableSink, uint64) { + *version += 1 + return innerTableSink, *version + }, + tablepb.TableStatePrepared, + oracle.GoTimeToTS(time.Now()), + oracle.GoTimeToTS(time.Now().Add(10000*time.Second)), + func(_ context.Context) (model.Ts, error) { return math.MaxUint64, nil }, + ) + + require.True(t, wrapper.initTableSink()) + + wrapper.closeAndClearTableSink() + + // Shouldn't be stuck because version is 0. + require.Equal(t, wrapper.tableSink.version, uint64(0)) + isStuck, _ := wrapper.sinkMaybeStuck(100 * time.Millisecond) + require.False(t, isStuck) + + // Shouldn't be stuck because tableSink.advanced is just updated. + require.True(t, wrapper.initTableSink()) + isStuck, _ = wrapper.sinkMaybeStuck(100 * time.Millisecond) + require.False(t, isStuck) + + // Shouldn't be stuck because upperbound hasn't been advanced. + time.Sleep(200 * time.Millisecond) + isStuck, _ = wrapper.sinkMaybeStuck(100 * time.Millisecond) + require.False(t, isStuck) + + // Shouldn't be stuck because `getCheckpointTs` will update tableSink.advanced. + nowTs := oracle.GoTimeToTS(time.Now()) + wrapper.updateReceivedSorterResolvedTs(nowTs) + wrapper.barrierTs.Store(nowTs) + isStuck, _ = wrapper.sinkMaybeStuck(100 * time.Millisecond) + require.False(t, isStuck) + + time.Sleep(200 * time.Millisecond) + nowTs = oracle.GoTimeToTS(time.Now()) + wrapper.updateReceivedSorterResolvedTs(nowTs) + wrapper.barrierTs.Store(nowTs) + wrapper.updateResolvedTs(model.NewResolvedTs(nowTs)) + isStuck, _ = wrapper.sinkMaybeStuck(100 * time.Millisecond) + require.True(t, isStuck) +} diff --git a/cdc/scheduler/internal/v3/agent/agent.go b/cdc/scheduler/internal/v3/agent/agent.go index e4dad8bb903..45390b9d244 100644 --- a/cdc/scheduler/internal/v3/agent/agent.go +++ b/cdc/scheduler/internal/v3/agent/agent.go @@ -206,10 +206,7 @@ func (a *agent) Tick(ctx context.Context) (*schedulepb.Barrier, error) { return nil, errors.Trace(err) } - outboundMessages, barrier, err := a.handleMessage(inboundMessages) - if err != nil { - return nil, errors.Trace(err) - } + outboundMessages, barrier := a.handleMessage(inboundMessages) responses, err := a.tableM.poll(ctx) if err != nil { @@ -237,9 +234,7 @@ func (a *agent) handleLivenessUpdate(liveness model.Liveness) { } } -func (a *agent) handleMessage(msg []*schedulepb.Message) ( - result []*schedulepb.Message, barrier *schedulepb.Barrier, err error, -) { +func (a *agent) handleMessage(msg []*schedulepb.Message) (result []*schedulepb.Message, barrier *schedulepb.Barrier) { for _, message := range msg { ownerCaptureID := message.GetFrom() header := message.GetHeader() @@ -254,10 +249,7 @@ func (a *agent) handleMessage(msg []*schedulepb.Message) ( switch message.GetMsgType() { case schedulepb.MsgHeartbeat: var reMsg *schedulepb.Message - reMsg, barrier, err = a.handleMessageHeartbeat(message.GetHeartbeat()) - if err != nil { - return - } + reMsg, barrier = a.handleMessageHeartbeat(message.GetHeartbeat()) result = append(result, reMsg) case schedulepb.MsgDispatchTableRequest: a.handleMessageDispatchTableRequest(message.DispatchTableRequest, processorEpoch) @@ -272,28 +264,24 @@ func (a *agent) handleMessage(msg []*schedulepb.Message) ( return } -func (a *agent) handleMessageHeartbeat(request *schedulepb.Heartbeat) ( - *schedulepb.Message, *schedulepb.Barrier, error, -) { +func (a *agent) handleMessageHeartbeat(request *schedulepb.Heartbeat) (*schedulepb.Message, *schedulepb.Barrier) { allTables := a.tableM.getAllTableSpans() result := make([]tablepb.TableStatus, 0, allTables.Len()) - isValidCheckpointTs := true allTables.Ascend(func(span tablepb.Span, table *tableSpan) bool { status := table.getTableSpanStatus(request.CollectStats) - isValidCheckpointTs = status.Checkpoint.CheckpointTs <= status.Checkpoint.ResolvedTs + if status.Checkpoint.CheckpointTs > status.Checkpoint.ResolvedTs { + log.Warn("schedulerv3: CheckpointTs is greater than ResolvedTs", + zap.String("namespace", a.ChangeFeedID.Namespace), + zap.String("changefeed", a.ChangeFeedID.ID), + zap.String("span", span.String())) + } if table.task != nil && table.task.IsRemove { status.State = tablepb.TableStateStopping } result = append(result, status) - return isValidCheckpointTs + return true }) - if !isValidCheckpointTs { - status := result[len(result)-1] - checkpointTs := status.Checkpoint.CheckpointTs - resolvedTs := status.Checkpoint.ResolvedTs - return nil, nil, errors.ErrInvalidCheckpointTs.GenWithStackByArgs(checkpointTs, resolvedTs) - } for _, span := range request.GetSpans() { if _, ok := allTables.Get(span); !ok { status := a.tableM.getTableSpanStatus(span, request.CollectStats) @@ -320,7 +308,7 @@ func (a *agent) handleMessageHeartbeat(request *schedulepb.Heartbeat) ( zap.String("changefeed", a.ChangeFeedID.ID), zap.Any("message", message)) - return message, request.GetBarrier(), nil + return message, request.GetBarrier() } type dispatchTableTaskStatus int32 diff --git a/cdc/scheduler/internal/v3/agent/agent_test.go b/cdc/scheduler/internal/v3/agent/agent_test.go index bc08d491f0c..1b9617b3d4d 100644 --- a/cdc/scheduler/internal/v3/agent/agent_test.go +++ b/cdc/scheduler/internal/v3/agent/agent_test.go @@ -356,7 +356,7 @@ func TestAgentHandleMessageHeartbeat(t *testing.T) { }, } - response, _, _ := a.handleMessage([]*schedulepb.Message{heartbeat}) + response, _ := a.handleMessage([]*schedulepb.Message{heartbeat}) require.Len(t, response, 1) require.Equal(t, model.LivenessCaptureAlive, response[0].GetHeartbeatResponse().Liveness) @@ -376,7 +376,7 @@ func TestAgentHandleMessageHeartbeat(t *testing.T) { } a.tableM.tables.GetV(spanz.TableIDToComparableSpan(1)).task = &dispatchTableTask{IsRemove: true} - response, _, _ = a.handleMessage([]*schedulepb.Message{heartbeat}) + response, _ = a.handleMessage([]*schedulepb.Message{heartbeat}) result = response[0].GetHeartbeatResponse().Tables sort.Slice(result, func(i, j int) bool { return result[i].Span.TableID < result[j].Span.TableID @@ -384,13 +384,13 @@ func TestAgentHandleMessageHeartbeat(t *testing.T) { require.Equal(t, tablepb.TableStateStopping, result[1].State) a.handleLivenessUpdate(model.LivenessCaptureStopping) - response, _, _ = a.handleMessage([]*schedulepb.Message{heartbeat}) + response, _ = a.handleMessage([]*schedulepb.Message{heartbeat}) require.Len(t, response, 1) require.Equal(t, model.LivenessCaptureStopping, response[0].GetHeartbeatResponse().Liveness) a.handleLivenessUpdate(model.LivenessCaptureAlive) heartbeat.Heartbeat.IsStopping = true - response, _, _ = a.handleMessage([]*schedulepb.Message{heartbeat}) + response, _ = a.handleMessage([]*schedulepb.Message{heartbeat}) require.Equal(t, model.LivenessCaptureStopping, response[0].GetHeartbeatResponse().Liveness) require.Equal(t, model.LivenessCaptureStopping, a.liveness.Load()) } @@ -577,7 +577,7 @@ func TestAgentHandleMessage(t *testing.T) { } // handle the first heartbeat, from the known owner. - response, _, _ := a.handleMessage([]*schedulepb.Message{heartbeat}) + response, _ := a.handleMessage([]*schedulepb.Message{heartbeat}) require.Len(t, response, 1) addTableRequest := &schedulepb.Message{ @@ -600,17 +600,17 @@ func TestAgentHandleMessage(t *testing.T) { }, } // wrong epoch, ignored - responses, _, _ := a.handleMessage([]*schedulepb.Message{addTableRequest}) + responses, _ := a.handleMessage([]*schedulepb.Message{addTableRequest}) require.False(t, tableM.tables.Has(spanz.TableIDToComparableSpan(1))) require.Len(t, responses, 0) // correct epoch, processing. addTableRequest.Header.ProcessorEpoch = a.Epoch - _, _, _ = a.handleMessage([]*schedulepb.Message{addTableRequest}) + _, _ = a.handleMessage([]*schedulepb.Message{addTableRequest}) require.True(t, a.tableM.tables.Has(spanz.TableIDToComparableSpan(1))) heartbeat.Header.OwnerRevision.Revision = 2 - response, _, _ = a.handleMessage([]*schedulepb.Message{heartbeat}) + response, _ = a.handleMessage([]*schedulepb.Message{heartbeat}) require.Len(t, response, 1) // this should never happen in real world @@ -624,12 +624,12 @@ func TestAgentHandleMessage(t *testing.T) { From: a.ownerInfo.ID, } - response, _, _ = a.handleMessage([]*schedulepb.Message{unknownMessage}) + response, _ = a.handleMessage([]*schedulepb.Message{unknownMessage}) require.Len(t, response, 0) // staled message heartbeat.Header.OwnerRevision.Revision = 1 - response, _, _ = a.handleMessage([]*schedulepb.Message{heartbeat}) + response, _ = a.handleMessage([]*schedulepb.Message{heartbeat}) require.Len(t, response, 0) } diff --git a/cdc/scheduler/internal/v3/keyspan/mock.go b/cdc/scheduler/internal/v3/keyspan/mock.go index a767508d17f..ffc5e58adf4 100644 --- a/cdc/scheduler/internal/v3/keyspan/mock.go +++ b/cdc/scheduler/internal/v3/keyspan/mock.go @@ -81,6 +81,6 @@ func NewReconcilerForTests( return &Reconciler{ tableSpans: make(map[int64]splittedSpans), config: config, - splitter: []splitter{newRegionCountSplitter(model.ChangeFeedID{}, cache)}, + splitter: []splitter{newRegionCountSplitter(model.ChangeFeedID{}, cache, config.RegionPerSpan)}, } } diff --git a/cdc/scheduler/internal/v3/keyspan/reconciler.go b/cdc/scheduler/internal/v3/keyspan/reconciler.go index 3dc115bef23..021cc28aa64 100644 --- a/cdc/scheduler/internal/v3/keyspan/reconciler.go +++ b/cdc/scheduler/internal/v3/keyspan/reconciler.go @@ -30,12 +30,17 @@ import ( "go.uber.org/zap" ) -const spanRegionLimit = 50000 +const ( + // spanRegionLimit is the maximum number of regions a span can cover. + spanRegionLimit = 50000 + // baseSpanNumberCoefficient is the base coefficient that use to + // multiply the number of captures to get the number of spans. + baseSpanNumberCoefficient = 3 +) type splitter interface { split( ctx context.Context, span tablepb.Span, totalCaptures int, - config *config.ChangefeedSchedulerConfig, ) []tablepb.Span } @@ -72,8 +77,8 @@ func NewReconciler( config: config, splitter: []splitter{ // write splitter has the highest priority. - newWriteSplitter(changefeedID, pdapi), - newRegionCountSplitter(changefeedID, up.RegionCache), + newWriteSplitter(changefeedID, pdapi, config.WriteKeyThreshold), + newRegionCountSplitter(changefeedID, up.RegionCache, config.RegionThreshold), }, }, nil } @@ -121,7 +126,7 @@ func (m *Reconciler) Reconcile( spans := []tablepb.Span{tableSpan} if compat.CheckSpanReplicationEnabled() { for _, splitter := range m.splitter { - spans = splitter.split(ctx, tableSpan, len(aliveCaptures), m.config) + spans = splitter.split(ctx, tableSpan, len(aliveCaptures)) if len(spans) > 1 { break } @@ -208,3 +213,24 @@ func (m *Reconciler) Reconcile( } return m.spanCache } + +const maxSpanNumber = 100 + +func getSpansNumber(regionNum, captureNum int) int { + coefficient := captureNum - 1 + if baseSpanNumberCoefficient > coefficient { + coefficient = baseSpanNumberCoefficient + } + spanNum := 1 + if regionNum > 1 { + // spanNumber = max(captureNum * coefficient, totalRegions / spanRegionLimit) + spanNum = captureNum * coefficient + if regionNum/spanRegionLimit > spanNum { + spanNum = regionNum / spanRegionLimit + } + } + if spanNum > maxSpanNumber { + spanNum = maxSpanNumber + } + return spanNum +} diff --git a/cdc/scheduler/internal/v3/keyspan/reconciler_test.go b/cdc/scheduler/internal/v3/keyspan/reconciler_test.go index 9da2aeef7b0..0552303dce7 100644 --- a/cdc/scheduler/internal/v3/keyspan/reconciler_test.go +++ b/cdc/scheduler/internal/v3/keyspan/reconciler_test.go @@ -259,3 +259,20 @@ func TestBatchAddRateLimit(t *testing.T) { require.Equal(t, allSpan, reconciler.tableSpans[2].spans) require.Equal(t, 1, len(reconciler.tableSpans)) } + +func TestGetSpansNumber(t *testing.T) { + tc := []struct { + regionCount int + captureNum int + expected int + }{ + {1, 10, 1}, + {100, 2, 6}, + {100, 3, 9}, + {100, 5, 20}, + {10000, 11, 100}, + } + for _, c := range tc { + require.Equal(t, c.expected, getSpansNumber(c.regionCount, c.captureNum)) + } +} diff --git a/cdc/scheduler/internal/v3/keyspan/splitter_region_count.go b/cdc/scheduler/internal/v3/keyspan/splitter_region_count.go index 74cd2ab760b..bc00ffa3d79 100644 --- a/cdc/scheduler/internal/v3/keyspan/splitter_region_count.go +++ b/cdc/scheduler/internal/v3/keyspan/splitter_region_count.go @@ -21,28 +21,28 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/processor/tablepb" - "github.com/pingcap/tiflow/pkg/config" "github.com/tikv/client-go/v2/tikv" "go.uber.org/zap" ) type regionCountSplitter struct { - changefeedID model.ChangeFeedID - regionCache RegionCache + changefeedID model.ChangeFeedID + regionCache RegionCache + regionThreshold int } func newRegionCountSplitter( - changefeedID model.ChangeFeedID, regionCache RegionCache, + changefeedID model.ChangeFeedID, regionCache RegionCache, regionThreshold int, ) *regionCountSplitter { return ®ionCountSplitter{ - changefeedID: changefeedID, - regionCache: regionCache, + changefeedID: changefeedID, + regionCache: regionCache, + regionThreshold: regionThreshold, } } func (m *regionCountSplitter) split( - ctx context.Context, span tablepb.Span, totalCaptures int, - config *config.ChangefeedSchedulerConfig, + ctx context.Context, span tablepb.Span, captureNum int, ) []tablepb.Span { bo := tikv.NewBackoffer(ctx, 500) regions, err := m.regionCache.ListRegionIDsInKeyRange(bo, span.StartKey, span.EndKey) @@ -54,29 +54,21 @@ func (m *regionCountSplitter) split( zap.Error(err)) return []tablepb.Span{span} } - if len(regions) <= config.RegionThreshold || totalCaptures == 0 { + if len(regions) <= m.regionThreshold || captureNum == 0 { log.Info("schedulerv3: skip split span by region count", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), zap.String("span", span.String()), - zap.Int("totalCaptures", totalCaptures), + zap.Int("totalCaptures", captureNum), zap.Int("regionCount", len(regions)), - zap.Int("regionThreshold", config.RegionThreshold)) + zap.Int("regionThreshold", m.regionThreshold)) return []tablepb.Span{span} } - pages := totalCaptures + stepper := newEvenlySplitStepper( + getSpansNumber(len(regions), captureNum), + len(regions)) - totalRegions := len(regions) - if totalRegions == 0 { - pages = 1 - } - - if totalRegions/spanRegionLimit > pages { - pages = totalRegions / spanRegionLimit - } - - stepper := newEvenlySplitStepper(pages, totalRegions) spans := make([]tablepb.Span, 0, stepper.SpanCount()) start, end := 0, stepper.Step() for { @@ -133,9 +125,9 @@ func (m *regionCountSplitter) split( zap.String("changefeed", m.changefeedID.ID), zap.String("span", span.String()), zap.Int("spans", len(spans)), - zap.Int("totalCaptures", totalCaptures), + zap.Int("totalCaptures", captureNum), zap.Int("regionCount", len(regions)), - zap.Int("regionThreshold", config.RegionThreshold), + zap.Int("regionThreshold", m.regionThreshold), zap.Int("spanRegionLimit", spanRegionLimit)) return spans } diff --git a/cdc/scheduler/internal/v3/keyspan/splitter_region_count_test.go b/cdc/scheduler/internal/v3/keyspan/splitter_region_count_test.go index 5063f1504bb..a42041dec58 100644 --- a/cdc/scheduler/internal/v3/keyspan/splitter_region_count_test.go +++ b/cdc/scheduler/internal/v3/keyspan/splitter_region_count_test.go @@ -77,7 +77,8 @@ func TestRegionCountSplitSpan(t *testing.T) { totalCaptures: 4, span: tablepb.Span{TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t2")}, expectSpans: []tablepb.Span{ - {TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t1_2")}, // 2 region + {TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t1_1")}, // 1 region + {TableID: 1, StartKey: []byte("t1_1"), EndKey: []byte("t1_2")}, // 1 region {TableID: 1, StartKey: []byte("t1_2"), EndKey: []byte("t1_3")}, // 1 region {TableID: 1, StartKey: []byte("t1_3"), EndKey: []byte("t1_4")}, // 1 region {TableID: 1, StartKey: []byte("t1_4"), EndKey: []byte("t2")}, // 1 region @@ -87,8 +88,10 @@ func TestRegionCountSplitSpan(t *testing.T) { totalCaptures: 3, span: tablepb.Span{TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t2")}, expectSpans: []tablepb.Span{ - {TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t1_2")}, // 2 region - {TableID: 1, StartKey: []byte("t1_2"), EndKey: []byte("t1_4")}, // 2 region + {TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t1_1")}, // 1 region + {TableID: 1, StartKey: []byte("t1_1"), EndKey: []byte("t1_2")}, // 1 region + {TableID: 1, StartKey: []byte("t1_2"), EndKey: []byte("t1_3")}, // 1 region + {TableID: 1, StartKey: []byte("t1_3"), EndKey: []byte("t1_4")}, // 1 region {TableID: 1, StartKey: []byte("t1_4"), EndKey: []byte("t2")}, // 1 region }, }, @@ -96,26 +99,31 @@ func TestRegionCountSplitSpan(t *testing.T) { totalCaptures: 2, span: tablepb.Span{TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t2")}, expectSpans: []tablepb.Span{ - {TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t1_3")}, // 3 region - {TableID: 1, StartKey: []byte("t1_3"), EndKey: []byte("t2")}, // 2 region + {TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t1_1")}, // 1 region + {TableID: 1, StartKey: []byte("t1_1"), EndKey: []byte("t1_2")}, // 1 region + {TableID: 1, StartKey: []byte("t1_2"), EndKey: []byte("t1_3")}, // 1 region + {TableID: 1, StartKey: []byte("t1_3"), EndKey: []byte("t1_4")}, // 1 region + {TableID: 1, StartKey: []byte("t1_4"), EndKey: []byte("t2")}, // 1 region }, }, { totalCaptures: 1, span: tablepb.Span{TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t2")}, expectSpans: []tablepb.Span{ - {TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t2")}, // 5 region + {TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t1_2")}, // 2 region + {TableID: 1, StartKey: []byte("t1_2"), EndKey: []byte("t1_4")}, // 2 region + {TableID: 1, StartKey: []byte("t1_4"), EndKey: []byte("t2")}, // 1 region }, }, } for i, cs := range cases { - splitter := newRegionCountSplitter(model.ChangeFeedID{}, cache) cfg := &config.ChangefeedSchedulerConfig{ EnableTableAcrossNodes: true, RegionThreshold: 1, } - spans := splitter.split(context.Background(), cs.span, cs.totalCaptures, cfg) + splitter := newRegionCountSplitter(model.ChangeFeedID{}, cache, cfg.RegionThreshold) + spans := splitter.split(context.Background(), cs.span, cs.totalCaptures) require.Equalf(t, cs.expectSpans, spans, "%d %s", i, &cs.span) } } @@ -134,64 +142,66 @@ func TestRegionCountEvenlySplitSpan(t *testing.T) { cases := []struct { totalCaptures int + expectedSpans int expectSpansMin int expectSpansMax int }{ { totalCaptures: 0, + expectedSpans: 1, expectSpansMin: 1000, expectSpansMax: 1000, }, { totalCaptures: 1, - expectSpansMin: 1000, - expectSpansMax: 1000, + expectedSpans: 3, + expectSpansMin: 333, + expectSpansMax: 334, }, { totalCaptures: 3, - expectSpansMin: 333, - expectSpansMax: 334, + expectedSpans: 9, + expectSpansMin: 111, + expectSpansMax: 113, }, { totalCaptures: 7, - expectSpansMin: 142, - expectSpansMax: 143, + expectedSpans: 42, + expectSpansMin: 23, + expectSpansMax: 24, }, { totalCaptures: 999, + expectedSpans: 100, expectSpansMin: 1, - expectSpansMax: 2, + expectSpansMax: 10, }, { totalCaptures: 1000, + expectedSpans: 100, expectSpansMin: 1, - expectSpansMax: 1, + expectSpansMax: 10, }, { totalCaptures: 2000, + expectedSpans: 100, expectSpansMin: 1, - expectSpansMax: 1, + expectSpansMax: 10, }, } for i, cs := range cases { - splitter := newRegionCountSplitter(model.ChangeFeedID{}, cache) cfg := &config.ChangefeedSchedulerConfig{ EnableTableAcrossNodes: true, RegionThreshold: 1, } + splitter := newRegionCountSplitter(model.ChangeFeedID{}, cache, cfg.RegionThreshold) spans := splitter.split( context.Background(), tablepb.Span{TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t2")}, cs.totalCaptures, - cfg, ) - if cs.totalCaptures == 0 { - require.Equalf(t, 1, len(spans), "%d %v", i, cs) - } else if cs.totalCaptures <= 1000 { - require.Equalf(t, cs.totalCaptures, len(spans), "%d %v", i, cs) - } else { - require.Equalf(t, 1000, len(spans), "%d %v", i, cs) - } + + require.Equalf(t, cs.expectedSpans, len(spans), "%d %v", i, cs) for _, span := range spans { start, end := 0, 1000 @@ -217,13 +227,13 @@ func TestSplitSpanRegionOutOfOrder(t *testing.T) { cache.regions.ReplaceOrInsert(tablepb.Span{StartKey: []byte("t1_1"), EndKey: []byte("t1_4")}, 2) cache.regions.ReplaceOrInsert(tablepb.Span{StartKey: []byte("t1_2"), EndKey: []byte("t1_3")}, 3) - splitter := newRegionCountSplitter(model.ChangeFeedID{}, cache) cfg := &config.ChangefeedSchedulerConfig{ EnableTableAcrossNodes: true, RegionThreshold: 1, } + splitter := newRegionCountSplitter(model.ChangeFeedID{}, cache, cfg.RegionThreshold) span := tablepb.Span{TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t2")} - spans := splitter.split(context.Background(), span, 1, cfg) + spans := splitter.split(context.Background(), span, 1) require.Equal( t, []tablepb.Span{{TableID: 1, StartKey: []byte("t1"), EndKey: []byte("t2")}}, spans) } diff --git a/cdc/scheduler/internal/v3/keyspan/splitter_write.go b/cdc/scheduler/internal/v3/keyspan/splitter_write.go index fc458ef788c..9773597b582 100644 --- a/cdc/scheduler/internal/v3/keyspan/splitter_write.go +++ b/cdc/scheduler/internal/v3/keyspan/splitter_write.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/processor/tablepb" - "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/pdutil" "go.uber.org/zap" ) @@ -28,24 +27,36 @@ import ( const regionWrittenKeyBase = 1 type writeSplitter struct { - changefeedID model.ChangeFeedID - pdAPIClient pdutil.PDAPIClient + changefeedID model.ChangeFeedID + pdAPIClient pdutil.PDAPIClient + writeKeyThreshold int +} + +type splitRegionsInfo struct { + RegionCounts []int + Weights []uint64 + WriteKeys []uint64 + Spans []tablepb.Span } func newWriteSplitter( - changefeedID model.ChangeFeedID, pdAPIClient pdutil.PDAPIClient, + changefeedID model.ChangeFeedID, + pdAPIClient pdutil.PDAPIClient, + writeKeyThreshold int, ) *writeSplitter { return &writeSplitter{ - changefeedID: changefeedID, - pdAPIClient: pdAPIClient, + changefeedID: changefeedID, + pdAPIClient: pdAPIClient, + writeKeyThreshold: writeKeyThreshold, } } func (m *writeSplitter) split( - ctx context.Context, span tablepb.Span, totalCaptures int, - config *config.ChangefeedSchedulerConfig, + ctx context.Context, + span tablepb.Span, + captureNum int, ) []tablepb.Span { - if config.WriteKeyThreshold == 0 { + if m.writeKeyThreshold == 0 { return nil } regions, err := m.pdAPIClient.ScanRegions(ctx, span) @@ -59,12 +70,8 @@ func (m *writeSplitter) split( return nil } - pages := totalCaptures - if len(regions)/spanRegionLimit > pages { - pages = len(regions) / spanRegionLimit - } - - if pages <= 1 { + spansNum := getSpansNumber(len(regions), captureNum) + if spansNum <= 1 { log.Warn("schedulerv3: only one capture and the regions number less than"+ " the maxSpanRegionLimit, skip split span", zap.String("namespace", m.changefeedID.Namespace), @@ -74,54 +81,60 @@ func (m *writeSplitter) split( return []tablepb.Span{span} } - info := splitRegionsByWrittenKeys(span.TableID, - regions, - config.WriteKeyThreshold, - pages, - spanRegionLimit) - + splitInfo := m.splitRegionsByWrittenKeysV1(span.TableID, regions, spansNum) log.Info("schedulerv3: split span by written keys", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), zap.String("span", span.String()), - zap.Ints("counts", info.Counts), - zap.Ints("weights", info.Weights), - zap.Int("spans", len(info.Spans)), - zap.Int("totalCaptures", totalCaptures), - zap.Int("writeKeyThreshold", config.WriteKeyThreshold), + zap.Ints("perSpanRegionCounts", splitInfo.RegionCounts), + zap.Uint64s("weights", splitInfo.Weights), + zap.Int("spans", len(splitInfo.Spans)), + zap.Int("totalCaptures", captureNum), + zap.Int("writeKeyThreshold", m.writeKeyThreshold), zap.Int("spanRegionLimit", spanRegionLimit)) - return info.Spans -} -type splitRegionsInfo struct { - Counts []int - Weights []int - Spans []tablepb.Span + return splitInfo.Spans } -// splitRegionsByWrittenKeys returns a slice of regions that evenly split the range by write keys. -// pages is the number of splits to make, actually it is the number of captures. -func splitRegionsByWrittenKeys( - tableID model.TableID, regions []pdutil.RegionInfo, - writeKeyThreshold int, pages int, spanRegionLimit int, +// splitRegionsByWrittenKeysV1 tries to split the regions into at least `baseSpansNum` spans, +// each span has approximately the same write weight. +// The algorithm is: +// 1. Sum the written keys of all regions, and normalize the written keys of each region by +// adding baseline weights (regionWrittenKeyBase) to each region's written keys. Which takes +// the region number into account. +// 2. Calculate the writeLimitPerSpan. +// 3. Split the table into spans: +// 3.1 If the total write is less than writeKeyThreshold, don't need to split the regions. +// 3.2 If the restSpans count is one, and the restWeight is less than writeLimitPerSpan, +// we will use the rest regions as the last span. If the restWeight is larger than writeLimitPerSpan, +// then we need to add more restSpans (restWeight / writeLimitPerSpan) to split the rest regions. +// 3.3 If the restRegions is less than equal to restSpans, then every region will be a span. +// 3.4 If the spanWriteWeight is larger than writeLimitPerSpan or the regionCount is larger +// than spanRegionLimit, then use the region range from spanStartIndex to i to as a span. +// 4. Return the split result. +func (m *writeSplitter) splitRegionsByWrittenKeysV1( + tableID model.TableID, + regions []pdutil.RegionInfo, + baseSpansNum int, ) *splitRegionsInfo { decodeKey := func(hexkey string) []byte { key, _ := hex.DecodeString(hexkey) return key } - totalWriteNormalized := uint64(0) - totalWrite := totalWriteNormalized + + totalWrite, totalWriteNormalized := uint64(0), uint64(0) for i := range regions { totalWrite += regions[i].WrittenKeys - // Override 0 to 1 to reflect the baseline cost of a region. - // Also, it makes split evenly when there is no write. regions[i].WrittenKeys += regionWrittenKeyBase totalWriteNormalized += regions[i].WrittenKeys } - if totalWrite < uint64(writeKeyThreshold) { + + // 1. If the total write is less than writeKeyThreshold + // don't need to split the regions + if totalWrite < uint64(m.writeKeyThreshold) { return &splitRegionsInfo{ - Counts: []int{len(regions)}, - Weights: []int{int(totalWriteNormalized)}, + RegionCounts: []int{len(regions)}, + Weights: []uint64{totalWriteNormalized}, Spans: []tablepb.Span{{ TableID: tableID, StartKey: tablepb.Key(decodeKey(regions[0].StartKey)), @@ -130,65 +143,104 @@ func splitRegionsByWrittenKeys( } } - writtenKeysPerPage := totalWriteNormalized / uint64(pages) - counts := make([]int, 0, pages) - weights := make([]int, 0, pages) - spans := make([]tablepb.Span, 0, pages) - accWrittenKeys, pageWrittenKeys := uint64(0), uint64(0) - pageStartIdx, pageLastIdx := 0, 0 - pageRegionsCount := 0 - // split the table into pages-1 spans, each span has writtenKeysPerPage written keys. - for i := 1; i < pages; i++ { - for idx := pageStartIdx; idx < len(regions); idx++ { - restPages := pages - i - restRegions := len(regions) - idx - pageLastIdx = idx - currentWrittenKeys := regions[idx].WrittenKeys - // If there is at least one region, and the rest regions can't fill the rest pages or - // the accWrittenKeys plus currentWrittenKeys is larger than writtenKeysPerPage, - // then use the region from pageStartIdx to idx-1 to as a span and start a new page. - if (idx > pageStartIdx) && - ((restPages >= restRegions) || - (accWrittenKeys+currentWrittenKeys > writtenKeysPerPage) || - pageRegionsCount >= spanRegionLimit) { + // 2. Calculate the writeLimitPerSpan, if one span's write is larger that + // this number, we should create a new span. + writeLimitPerSpan := totalWriteNormalized / uint64(baseSpansNum) + + // The result of this method + var ( + regionCounts = make([]int, 0, baseSpansNum) + writeKeys = make([]uint64, 0, baseSpansNum) + weights = make([]uint64, 0, baseSpansNum) + spans = make([]tablepb.Span, 0, baseSpansNum) + ) + + // Temp variables used in the loop + var ( + spanWriteWeight = uint64(0) + spanStartIndex = 0 + restSpans = baseSpansNum + regionCount = 0 + restWeight = int64(totalWriteNormalized) + ) + + // 3. Split the table into spans, each span has approximately + // `writeWeightPerSpan` weight or `spanRegionLimit` regions. + for i := 0; i < len(regions); i++ { + restRegions := len(regions) - i + regionCount++ + spanWriteWeight += regions[i].WrittenKeys + // If the restSpans count is one, and the restWeight is less than writeLimitPerSpan, + // we will use the rest regions as the last span. If the restWeight is larger than writeLimitPerSpan, + // then we need to add more restSpans (restWeight / writeLimitPerSpan) to split the rest regions. + if restSpans == 1 { + if restWeight < int64(writeLimitPerSpan) { spans = append(spans, tablepb.Span{ TableID: tableID, - StartKey: tablepb.Key(decodeKey(regions[pageStartIdx].StartKey)), - EndKey: tablepb.Key(decodeKey(regions[idx-1].EndKey)), + StartKey: tablepb.Key(decodeKey(regions[spanStartIndex].StartKey)), + EndKey: tablepb.Key(decodeKey(regions[len(regions)-1].EndKey)), }) - counts = append(counts, idx-pageStartIdx) - weights = append(weights, int(pageWrittenKeys)) - pageWrittenKeys = 0 - pageStartIdx = idx - // update writtenKeysPerPage to make the rest regions evenly split - // to the rest pages. - writtenKeysPerPage = (totalWriteNormalized - accWrittenKeys) / uint64(restPages) - accWrittenKeys = 0 - pageRegionsCount = 0 + + lastSpanRegionCount := len(regions) - spanStartIndex + lastSpanWriteWeight := uint64(0) + lastSpanWriteKey := uint64(0) + for j := spanStartIndex; j < len(regions); j++ { + lastSpanWriteKey += regions[j].WrittenKeys + lastSpanWriteWeight += regions[j].WrittenKeys + } + regionCounts = append(regionCounts, lastSpanRegionCount) + weights = append(weights, lastSpanWriteWeight) + writeKeys = append(writeKeys, lastSpanWriteKey) break } - pageWrittenKeys += currentWrittenKeys - accWrittenKeys += currentWrittenKeys - pageRegionsCount++ + // If the restWeight is larger than writeLimitPerSpan, + // then we need to update the restSpans. + restSpans = int(restWeight) / int(writeLimitPerSpan) } - } - // The last span contains the rest regions. - spans = append(spans, tablepb.Span{ - TableID: tableID, - StartKey: tablepb.Key(decodeKey(regions[pageLastIdx].StartKey)), - EndKey: tablepb.Key(decodeKey(regions[len(regions)-1].EndKey)), - }) - counts = append(counts, len(regions)-pageLastIdx) - pageWrittenKeys = 0 - for idx := pageLastIdx; idx < len(regions); idx++ { - pageWrittenKeys += regions[idx].WrittenKeys - } - weights = append(weights, int(pageWrittenKeys)) + // If the restRegions is less than equal to restSpans, + // then every region will be a span. + if restRegions <= restSpans { + spans = append(spans, tablepb.Span{ + TableID: tableID, + StartKey: tablepb.Key(decodeKey(regions[spanStartIndex].StartKey)), + EndKey: tablepb.Key(decodeKey(regions[i].EndKey)), + }) + regionCounts = append(regionCounts, regionCount) + weights = append(weights, spanWriteWeight) + + // reset the temp variables to start a new span + restSpans-- + restWeight -= int64(spanWriteWeight) + spanWriteWeight = 0 + regionCount = 0 + spanStartIndex = i + 1 + continue + } + // If the spanWriteWeight is larger than writeLimitPerSpan or the regionCount + // is larger than spanRegionLimit, then use the region range from + // spanStartIndex to i to as a span. + if spanWriteWeight > writeLimitPerSpan || regionCount >= spanRegionLimit { + spans = append(spans, tablepb.Span{ + TableID: tableID, + StartKey: tablepb.Key(decodeKey(regions[spanStartIndex].StartKey)), + EndKey: tablepb.Key(decodeKey(regions[i].EndKey)), + }) + regionCounts = append(regionCounts, regionCount) + weights = append(weights, spanWriteWeight) + // reset the temp variables to start a new span + restSpans-- + restWeight -= int64(spanWriteWeight) + spanWriteWeight = 0 + regionCount = 0 + spanStartIndex = i + 1 + } + } return &splitRegionsInfo{ - Counts: counts, - Weights: weights, - Spans: spans, + RegionCounts: regionCounts, + Weights: weights, + WriteKeys: writeKeys, + Spans: spans, } } diff --git a/cdc/scheduler/internal/v3/keyspan/splitter_write_test.go b/cdc/scheduler/internal/v3/keyspan/splitter_write_test.go index 56590644435..2987857c776 100644 --- a/cdc/scheduler/internal/v3/keyspan/splitter_write_test.go +++ b/cdc/scheduler/internal/v3/keyspan/splitter_write_test.go @@ -17,10 +17,12 @@ import ( "context" "encoding/hex" "math" + "math/rand" + "strconv" "testing" + "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/processor/tablepb" - "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/pdutil" "github.com/stretchr/testify/require" ) @@ -57,50 +59,50 @@ func TestSplitRegionsByWrittenKeysUniform(t *testing.T) { re := require.New(t) regions, startKeys, endKeys := prepareRegionsInfo( - [7]int{100, 100, 100, 100, 100, 100, 100}) - - info := splitRegionsByWrittenKeys(0, cloneRegions(regions), 0, 1, spanRegionLimit) - re.Len(info.Counts, 1) - re.EqualValues(7, info.Counts[0]) + [7]int{100, 100, 100, 100, 100, 100, 100}) // region id: [2,3,4,5,6,7,8] + splitter := newWriteSplitter(model.ChangeFeedID4Test("test", "test"), nil, 0) + info := splitter.splitRegionsByWrittenKeysV1(0, cloneRegions(regions), 1) + re.Len(info.RegionCounts, 1) + re.EqualValues(7, info.RegionCounts[0]) re.Len(info.Spans, 1) re.EqualValues(startKeys[2], info.Spans[0].StartKey) re.EqualValues(endKeys[8], info.Spans[0].EndKey) - info = splitRegionsByWrittenKeys(0, cloneRegions(regions), 0, 2, spanRegionLimit) // [2,3,4], [5,6,7,8] - re.Len(info.Counts, 2) - re.EqualValues(3, info.Counts[0]) - re.EqualValues(4, info.Counts[1]) + info = splitter.splitRegionsByWrittenKeysV1(0, cloneRegions(regions), 2) // [2,3,4,5], [6,7,8] + re.Len(info.RegionCounts, 2) + re.EqualValues(4, info.RegionCounts[0]) + re.EqualValues(3, info.RegionCounts[1]) re.Len(info.Weights, 2) - re.EqualValues(303, info.Weights[0]) - re.EqualValues(404, info.Weights[1]) + re.EqualValues(404, info.Weights[0]) + re.EqualValues(303, info.Weights[1]) re.Len(info.Spans, 2) re.EqualValues(startKeys[2], info.Spans[0].StartKey) - re.EqualValues(endKeys[4], info.Spans[0].EndKey) - re.EqualValues(startKeys[5], info.Spans[1].StartKey) + re.EqualValues(endKeys[5], info.Spans[0].EndKey) + re.EqualValues(startKeys[6], info.Spans[1].StartKey) re.EqualValues(endKeys[8], info.Spans[1].EndKey) - info = splitRegionsByWrittenKeys(0, cloneRegions(regions), 0, 3, spanRegionLimit) // [2,3], [4,5,6], [7,8] - re.Len(info.Counts, 3) - re.EqualValues(2, info.Counts[0]) - re.EqualValues(2, info.Counts[1]) - re.EqualValues(3, info.Counts[2]) + info = splitter.splitRegionsByWrittenKeysV1(0, cloneRegions(regions), 3) // [2,3,4], [5,6,7], [8] + re.Len(info.RegionCounts, 3) + re.EqualValues(3, info.RegionCounts[0]) + re.EqualValues(3, info.RegionCounts[1]) + re.EqualValues(1, info.RegionCounts[2]) re.Len(info.Weights, 3) - re.EqualValues(202, info.Weights[0]) - re.EqualValues(202, info.Weights[1]) - re.EqualValues(303, info.Weights[2]) + re.EqualValues(303, info.Weights[0]) + re.EqualValues(303, info.Weights[1]) + re.EqualValues(101, info.Weights[2]) re.Len(info.Spans, 3) re.EqualValues(startKeys[2], info.Spans[0].StartKey) - re.EqualValues(endKeys[3], info.Spans[0].EndKey) - re.EqualValues(startKeys[4], info.Spans[1].StartKey) - re.EqualValues(endKeys[5], info.Spans[1].EndKey) - re.EqualValues(startKeys[6], info.Spans[2].StartKey) + re.EqualValues(endKeys[4], info.Spans[0].EndKey) + re.EqualValues(startKeys[5], info.Spans[1].StartKey) + re.EqualValues(endKeys[7], info.Spans[1].EndKey) + re.EqualValues(startKeys[8], info.Spans[2].StartKey) re.EqualValues(endKeys[8], info.Spans[2].EndKey) - // Pages > regons + // spans > regions for p := 7; p <= 10; p++ { - info = splitRegionsByWrittenKeys(0, cloneRegions(regions), 0, p, spanRegionLimit) - re.Len(info.Counts, 7) - for _, c := range info.Counts { + info = splitter.splitRegionsByWrittenKeysV1(0, cloneRegions(regions), p) + re.Len(info.RegionCounts, 7) + for _, c := range info.RegionCounts { re.EqualValues(1, c) } re.Len(info.Weights, 7) @@ -113,11 +115,6 @@ func TestSplitRegionsByWrittenKeysUniform(t *testing.T) { re.EqualValues(endKeys[2+i], r.EndKey) } } - - // test spanRegionLimit works - info = splitRegionsByWrittenKeys(0, cloneRegions(regions), 0, 2, 3) - re.Len(info.Counts, 2) - re.EqualValues(3, info.Counts[0]) } func TestSplitRegionsByWrittenKeysHotspot1(t *testing.T) { @@ -127,24 +124,24 @@ func TestSplitRegionsByWrittenKeysHotspot1(t *testing.T) { // Hotspots regions, startKeys, endKeys := prepareRegionsInfo( [7]int{100, 1, 100, 1, 1, 1, 100}) - - info := splitRegionsByWrittenKeys(0, regions, 0, 4, spanRegionLimit) // [2], [3,4], [5,6,7], [8] - re.Len(info.Counts, 4) - re.EqualValues(1, info.Counts[0]) - re.EqualValues(1, info.Counts[1]) - re.EqualValues(4, info.Counts[2]) - re.EqualValues(1, info.Counts[3]) + splitter := newWriteSplitter(model.ChangeFeedID4Test("test", "test"), nil, 4) + info := splitter.splitRegionsByWrittenKeysV1(0, regions, 4) // [2], [3,4], [5,6,7], [8] + re.Len(info.RegionCounts, 4) + re.EqualValues(1, info.RegionCounts[0]) + re.EqualValues(2, info.RegionCounts[1]) + re.EqualValues(3, info.RegionCounts[2]) + re.EqualValues(1, info.RegionCounts[3]) re.Len(info.Weights, 4) re.EqualValues(101, info.Weights[0]) - re.EqualValues(2, info.Weights[1]) - re.EqualValues(107, info.Weights[2]) + re.EqualValues(103, info.Weights[1]) + re.EqualValues(6, info.Weights[2]) re.EqualValues(101, info.Weights[3]) re.Len(info.Spans, 4) re.EqualValues(startKeys[2], info.Spans[0].StartKey) re.EqualValues(endKeys[2], info.Spans[0].EndKey) re.EqualValues(startKeys[3], info.Spans[1].StartKey) - re.EqualValues(endKeys[3], info.Spans[1].EndKey) - re.EqualValues(startKeys[4], info.Spans[2].StartKey) + re.EqualValues(endKeys[4], info.Spans[1].EndKey) + re.EqualValues(startKeys[5], info.Spans[2].StartKey) re.EqualValues(endKeys[7], info.Spans[2].EndKey) re.EqualValues(startKeys[8], info.Spans[3].StartKey) re.EqualValues(endKeys[8], info.Spans[3].EndKey) @@ -157,14 +154,14 @@ func TestSplitRegionsByWrittenKeysHotspot2(t *testing.T) { // Hotspots regions, startKeys, endKeys := prepareRegionsInfo( [7]int{1000, 1, 1, 1, 100, 1, 99}) - - info := splitRegionsByWrittenKeys(0, regions, 0, 4, spanRegionLimit) // [2], [3,4,5], [6,7], [8] + splitter := newWriteSplitter(model.ChangeFeedID4Test("test", "test"), nil, 4) + info := splitter.splitRegionsByWrittenKeysV1(0, regions, 4) // [2], [3,4,5,6], [7], [8] re.Len(info.Spans, 4) re.EqualValues(startKeys[2], info.Spans[0].StartKey) re.EqualValues(endKeys[2], info.Spans[0].EndKey) re.EqualValues(startKeys[3], info.Spans[1].StartKey) - re.EqualValues(endKeys[5], info.Spans[1].EndKey) - re.EqualValues(startKeys[6], info.Spans[2].StartKey) + re.EqualValues(endKeys[6], info.Spans[1].EndKey) + re.EqualValues(startKeys[7], info.Spans[2].StartKey) re.EqualValues(endKeys[7], info.Spans[2].EndKey) re.EqualValues(startKeys[8], info.Spans[3].StartKey) re.EqualValues(endKeys[8], info.Spans[3].EndKey) @@ -173,23 +170,23 @@ func TestSplitRegionsByWrittenKeysHotspot2(t *testing.T) { func TestSplitRegionsByWrittenKeysCold(t *testing.T) { t.Parallel() re := require.New(t) - + splitter := newWriteSplitter(model.ChangeFeedID4Test("test", "test"), nil, 0) regions, startKeys, endKeys := prepareRegionsInfo([7]int{}) - info := splitRegionsByWrittenKeys(0, regions, 0, 3, spanRegionLimit) // [2,3], [4,5], [6,7,8] - re.Len(info.Counts, 3) - re.EqualValues(2, info.Counts[0], info) - re.EqualValues(2, info.Counts[1]) - re.EqualValues(3, info.Counts[2]) + info := splitter.splitRegionsByWrittenKeysV1(0, regions, 3) // [2,3,4], [5,6,7], [8] + re.Len(info.RegionCounts, 3) + re.EqualValues(3, info.RegionCounts[0], info) + re.EqualValues(3, info.RegionCounts[1]) + re.EqualValues(1, info.RegionCounts[2]) re.Len(info.Weights, 3) - re.EqualValues(2, info.Weights[0]) - re.EqualValues(2, info.Weights[1]) - re.EqualValues(3, info.Weights[2]) + re.EqualValues(3, info.Weights[0]) + re.EqualValues(3, info.Weights[1]) + re.EqualValues(1, info.Weights[2]) re.Len(info.Spans, 3) re.EqualValues(startKeys[2], info.Spans[0].StartKey) - re.EqualValues(endKeys[3], info.Spans[0].EndKey) - re.EqualValues(startKeys[4], info.Spans[1].StartKey) - re.EqualValues(endKeys[5], info.Spans[1].EndKey) - re.EqualValues(startKeys[6], info.Spans[2].StartKey) + re.EqualValues(endKeys[4], info.Spans[0].EndKey) + re.EqualValues(startKeys[5], info.Spans[1].StartKey) + re.EqualValues(endKeys[7], info.Spans[1].EndKey) + re.EqualValues(startKeys[8], info.Spans[2].StartKey) re.EqualValues(endKeys[8], info.Spans[2].EndKey) } @@ -197,10 +194,11 @@ func TestSplitRegionsByWrittenKeysConfig(t *testing.T) { t.Parallel() re := require.New(t) + splitter := newWriteSplitter(model.ChangeFeedID4Test("test", "test"), nil, math.MaxInt) regions, startKeys, endKeys := prepareRegionsInfo([7]int{1, 1, 1, 1, 1, 1, 1}) - info := splitRegionsByWrittenKeys(1, regions, math.MaxInt, 3, spanRegionLimit) // [2,3,4,5,6,7,8] - re.Len(info.Counts, 1) - re.EqualValues(7, info.Counts[0], info) + info := splitter.splitRegionsByWrittenKeysV1(1, regions, 3) // [2,3,4,5,6,7,8] + re.Len(info.RegionCounts, 1) + re.EqualValues(7, info.RegionCounts[0], info) re.Len(info.Weights, 1) re.EqualValues(14, info.Weights[0]) re.Len(info.Spans, 1) @@ -208,9 +206,119 @@ func TestSplitRegionsByWrittenKeysConfig(t *testing.T) { re.EqualValues(endKeys[8], info.Spans[0].EndKey) re.EqualValues(1, info.Spans[0].TableID) - s := writeSplitter{} - spans := s.split(context.Background(), tablepb.Span{}, 3, &config.ChangefeedSchedulerConfig{ - WriteKeyThreshold: 0, - }) + splitter.writeKeyThreshold = 0 + spans := splitter.split(context.Background(), tablepb.Span{}, 3) require.Empty(t, spans) } + +func TestSplitRegionEven(t *testing.T) { + tblID := model.TableID(1) + regionCount := 4653 + 1051 + 745 + 9530 + 1 + regions := make([]pdutil.RegionInfo, regionCount) + for i := 0; i < regionCount; i++ { + regions[i] = pdutil.RegionInfo{ + ID: uint64(i), + StartKey: "" + strconv.Itoa(i), + EndKey: "" + strconv.Itoa(i), + WrittenKeys: 2, + } + } + splitter := newWriteSplitter(model.ChangeFeedID4Test("test", "test"), nil, 4) + info := splitter.splitRegionsByWrittenKeysV1(tblID, regions, 5) + require.Len(t, info.RegionCounts, 5) + require.Len(t, info.Weights, 5) + for i, w := range info.Weights { + if i == 4 { + require.Equal(t, uint64(9576), w, i) + } else { + require.Equal(t, uint64(9591), w, i) + } + } +} + +func TestSpanRegionLimitBase(t *testing.T) { + splitter := newWriteSplitter(model.ChangeFeedID4Test("test", "test"), nil, 0) + var regions []pdutil.RegionInfo + // test spanRegionLimit works + for i := 0; i < spanRegionLimit*6; i++ { + regions = append(regions, pdutil.NewTestRegionInfo(uint64(i+9), []byte("f"), []byte("f"), 100)) + } + captureNum := 2 + spanNum := getSpansNumber(len(regions), captureNum) + info := splitter.splitRegionsByWrittenKeysV1(0, cloneRegions(regions), spanNum) + require.Len(t, info.RegionCounts, spanNum) + for _, c := range info.RegionCounts { + require.LessOrEqual(t, c, int(spanRegionLimit*1.1)) + } +} + +func TestSpanRegionLimit(t *testing.T) { + // Fisher-Yates shuffle algorithm to shuffle the writtenKeys + // but keep the first preservationRate% of the writtenKeys in the left side of the list + // to make the writtenKeys more like a hot region list + shuffle := func(nums []int, preservationRate float64) []int { + n := len(nums) + shuffled := make([]int, n) + copy(shuffled, nums) + + for i := n - 1; i > 0; i-- { + if rand.Float64() < preservationRate { + continue + } + j := rand.Intn(i + 1) + shuffled[i], shuffled[j] = shuffled[j], shuffled[i] + } + + return shuffled + } + + // total region number + totalRegionNumbers := spanRegionLimit * 10 + + // writtenKeys over 20000 percentage + percentOver20000 := 1 + // writtenKeys between 5000 and 10000 percentage + percentBetween5000And10000 := 5 + + countOver20000 := (percentOver20000 * totalRegionNumbers) / 100 + countBetween5000And10000 := (percentBetween5000And10000 * totalRegionNumbers) / 100 + countBelow1000 := totalRegionNumbers - countOver20000 - countBetween5000And10000 + + // random generate writtenKeys for each region + var writtenKeys []int + + for i := 0; i < countOver20000; i++ { + number := rand.Intn(80000) + 20001 + writtenKeys = append(writtenKeys, number) + } + + for i := 0; i < countBetween5000And10000; i++ { + number := rand.Intn(5001) + 5000 + writtenKeys = append(writtenKeys, number) + } + + for i := 0; i < countBelow1000; i++ { + number := rand.Intn(1000) + writtenKeys = append(writtenKeys, number) + } + + // 70% hot written region is in the left side of the region list + writtenKeys = shuffle(writtenKeys, 0.7) + + splitter := newWriteSplitter(model.ChangeFeedID4Test("test", "test"), nil, 0) + var regions []pdutil.RegionInfo + // region number is 500,000 + // weight is random between 0 and 40,000 + for i := 0; i < len(writtenKeys); i++ { + regions = append( + regions, + pdutil.NewTestRegionInfo(uint64(i+9), []byte("f"), []byte("f"), uint64(writtenKeys[i]))) + } + captureNum := 3 + spanNum := getSpansNumber(len(regions), captureNum) + info := splitter.splitRegionsByWrittenKeysV1(0, cloneRegions(regions), spanNum) + require.LessOrEqual(t, spanNum, len(info.RegionCounts)) + for _, c := range info.RegionCounts { + require.LessOrEqual(t, c, int(spanRegionLimit*1.1)) + } +} diff --git a/cdc/scheduler/internal/v3/replication/replication_manager.go b/cdc/scheduler/internal/v3/replication/replication_manager.go index fe98b3a41f2..9a33d460bbf 100644 --- a/cdc/scheduler/internal/v3/replication/replication_manager.go +++ b/cdc/scheduler/internal/v3/replication/replication_manager.go @@ -512,6 +512,38 @@ func (r *Manager) AdvanceCheckpoint( barrier *schedulepb.BarrierWithMinTs, redoMetaManager redo.MetaManager, ) (newCheckpointTs, newResolvedTs model.Ts) { + var redoFlushedResolvedTs model.Ts + limitBarrierWithRedo := func(newCheckpointTs, newResolvedTs uint64) (uint64, uint64) { + flushedMeta := redoMetaManager.GetFlushedMeta() + redoFlushedResolvedTs = flushedMeta.ResolvedTs + log.Debug("owner gets flushed redo meta", + zap.String("namespace", r.changefeedID.Namespace), + zap.String("changefeed", r.changefeedID.ID), + zap.Uint64("flushedCheckpointTs", flushedMeta.CheckpointTs), + zap.Uint64("flushedResolvedTs", flushedMeta.ResolvedTs)) + if flushedMeta.ResolvedTs < newResolvedTs { + newResolvedTs = flushedMeta.ResolvedTs + } + + if newCheckpointTs > newResolvedTs { + newCheckpointTs = newResolvedTs + } + + if barrier.GlobalBarrierTs > newResolvedTs { + barrier.GlobalBarrierTs = newResolvedTs + } + return newCheckpointTs, newResolvedTs + } + defer func() { + if redoFlushedResolvedTs != 0 && barrier.GlobalBarrierTs > redoFlushedResolvedTs { + log.Panic("barrierTs should never greater than redo flushed", + zap.String("namespace", r.changefeedID.Namespace), + zap.String("changefeed", r.changefeedID.ID), + zap.Uint64("barrierTs", barrier.GlobalBarrierTs), + zap.Uint64("redoFlushedResolvedTs", redoFlushedResolvedTs)) + } + }() + newCheckpointTs, newResolvedTs = math.MaxUint64, math.MaxUint64 slowestRange := tablepb.Span{} cannotProceed := false @@ -573,6 +605,11 @@ func (r *Manager) AdvanceCheckpoint( return true }) if cannotProceed { + if redoMetaManager.Enabled() { + // If redo is enabled, GlobalBarrierTs should be limited by redo flushed meta. + newResolvedTs = barrier.RedoBarrierTs + limitBarrierWithRedo(newCheckpointTs, newResolvedTs) + } return checkpointCannotProceed, checkpointCannotProceed } if slowestRange.TableID != 0 { @@ -619,26 +656,12 @@ func (r *Manager) AdvanceCheckpoint( newResolvedTs = barrier.RedoBarrierTs } redoMetaManager.UpdateMeta(newCheckpointTs, newResolvedTs) - flushedMeta := redoMetaManager.GetFlushedMeta() - flushedCheckpointTs, flushedResolvedTs := flushedMeta.CheckpointTs, flushedMeta.ResolvedTs - log.Debug("owner gets flushed meta", - zap.Uint64("flushedResolvedTs", flushedResolvedTs), - zap.Uint64("flushedCheckpointTs", flushedCheckpointTs), - zap.Uint64("newResolvedTs", newResolvedTs), - zap.Uint64("newCheckpointTs", newCheckpointTs), + log.Debug("owner updates redo meta", zap.String("namespace", r.changefeedID.Namespace), - zap.String("changefeed", r.changefeedID.ID)) - if flushedResolvedTs != 0 && flushedResolvedTs < newResolvedTs { - newResolvedTs = flushedResolvedTs - } - - if newCheckpointTs > newResolvedTs { - newCheckpointTs = newResolvedTs - } - - if barrier.GlobalBarrierTs > newResolvedTs { - barrier.GlobalBarrierTs = newResolvedTs - } + zap.String("changefeed", r.changefeedID.ID), + zap.Uint64("newCheckpointTs", newCheckpointTs), + zap.Uint64("newResolvedTs", newResolvedTs)) + return limitBarrierWithRedo(newCheckpointTs, newResolvedTs) } return newCheckpointTs, newResolvedTs diff --git a/cdc/scheduler/internal/v3/replication/replication_manager_test.go b/cdc/scheduler/internal/v3/replication/replication_manager_test.go index 692844659c2..0584e62972c 100644 --- a/cdc/scheduler/internal/v3/replication/replication_manager_test.go +++ b/cdc/scheduler/internal/v3/replication/replication_manager_test.go @@ -766,6 +766,51 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { require.Equal(t, model.Ts(9), barrier.GetGlobalBarrierTs()) } +func TestReplicationManagerAdvanceCheckpointWithRedoEnabled(t *testing.T) { + t.Parallel() + r := NewReplicationManager(1, model.ChangeFeedID{}) + span := spanz.TableIDToComparableSpan(1) + rs, err := NewReplicationSet(span, model.Ts(10), + map[model.CaptureID]*tablepb.TableStatus{ + "1": { + Span: spanz.TableIDToComparableSpan(1), + State: tablepb.TableStateReplicating, + Checkpoint: tablepb.Checkpoint{ + CheckpointTs: model.Ts(10), + ResolvedTs: model.Ts(20), + }, + }, + }, model.ChangeFeedID{}) + require.NoError(t, err) + r.spans.ReplaceOrInsert(span, rs) + + span2 := spanz.TableIDToComparableSpan(2) + rs, err = NewReplicationSet(span2, model.Ts(15), + map[model.CaptureID]*tablepb.TableStatus{ + "2": { + Span: spanz.TableIDToComparableSpan(2), + State: tablepb.TableStateReplicating, + Checkpoint: tablepb.Checkpoint{ + CheckpointTs: model.Ts(15), + ResolvedTs: model.Ts(30), + }, + }, + }, model.ChangeFeedID{}) + require.NoError(t, err) + r.spans.ReplaceOrInsert(span2, rs) + + redoMetaManager := &mockRedoMetaManager{enable: true, resolvedTs: 25} + + // some table not exist yet with redo is enabled. + currentTables := &TableRanges{} + currentTables.UpdateTables([]model.TableID{1, 2, 3}) + barrier := schedulepb.NewBarrierWithMinTs(30) + checkpoint, resolved := r.AdvanceCheckpoint(currentTables, time.Now(), barrier, redoMetaManager) + require.Equal(t, checkpointCannotProceed, checkpoint) + require.Equal(t, checkpointCannotProceed, resolved) + require.Equal(t, uint64(25), barrier.Barrier.GetGlobalBarrierTs()) +} + func TestReplicationManagerHandleCaptureChanges(t *testing.T) { t.Parallel() diff --git a/cdc/server/server.go b/cdc/server/server.go index c4523e4bb1b..98f28e087b7 100644 --- a/cdc/server/server.go +++ b/cdc/server/server.go @@ -42,15 +42,11 @@ import ( "github.com/pingcap/tiflow/pkg/util" p2pProto "github.com/pingcap/tiflow/proto/p2p" pd "github.com/tikv/pd/client" - "go.etcd.io/etcd/client/pkg/v3/logutil" - clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" - "go.uber.org/zap/zapcore" "golang.org/x/net/netutil" "golang.org/x/sync/errgroup" "google.golang.org/grpc" "google.golang.org/grpc/backoff" - "google.golang.org/grpc/keepalive" ) const ( @@ -78,11 +74,15 @@ type Server interface { // TODO: we need to make server more unit testable and add more test cases. // Especially we need to decouple the HTTPServer out of server. type server struct { - capture capture.Capture - tcpServer tcpserver.TCPServer - grpcService *p2p.ServerWrapper - statusServer *http.Server - etcdClient etcd.CDCEtcdClient + capture capture.Capture + tcpServer tcpserver.TCPServer + grpcService *p2p.ServerWrapper + statusServer *http.Server + etcdClient etcd.CDCEtcdClient + // pdClient is the default upstream PD client. + // The PD acts as a metadata management service for TiCDC. + pdClient pd.Client + pdAPIClient pdutil.PDAPIClient pdEndpoints []string sortEngineFactory *factory.SortEngineFactory } @@ -125,35 +125,21 @@ func New(pdEndpoints []string) (*server, error) { func (s *server) prepare(ctx context.Context) error { conf := config.GetGlobalServerConfig() - grpcTLSOption, err := conf.Security.ToGRPCDialOption() + tlsConfig, err := conf.Security.ToTLSConfig() if err != nil { return errors.Trace(err) } - - tlsConfig, err := conf.Security.ToTLSConfig() + grpcTLSOption, err := conf.Security.ToGRPCDialOption() if err != nil { return errors.Trace(err) } - - logConfig := logutil.DefaultZapLoggerConfig - logConfig.Level = zap.NewAtomicLevelAt(zapcore.ErrorLevel) - - log.Info("create etcdCli", zap.Strings("endpoints", s.pdEndpoints)) - // we do not pass a `context` to the etcd client, - // to prevent it's cancelled when the server is closing. - // For example, when the non-owner node goes offline, - // it would resign the campaign key which was put by call `campaign`, - // if this is not done due to the passed context cancelled, - // the key will be kept for the lease TTL, which is 10 seconds, - // then cause the new owner cannot be elected immediately after the old owner offline. - // see https://github.com/etcd-io/etcd/blob/525d53bd41/client/v3/concurrency/election.go#L98 - etcdCli, err := clientv3.New(clientv3.Config{ - Endpoints: s.pdEndpoints, - TLS: tlsConfig, - LogConfig: &logConfig, - DialTimeout: 5 * time.Second, - AutoSyncInterval: 30 * time.Second, - DialOptions: []grpc.DialOption{ + log.Info("create pd client", zap.Strings("endpoints", s.pdEndpoints)) + s.pdClient, err = pd.NewClientWithContext( + ctx, s.pdEndpoints, conf.Security.PDSecurityOption(), + // the default `timeout` is 3s, maybe too small if the pd is busy, + // set to 10s to avoid frequent timeout. + pd.WithCustomTimeoutOption(10*time.Second), + pd.WithGRPCDialOptions( grpcTLSOption, grpc.WithBlock(), grpc.WithConnectParams(grpc.ConnectParams{ @@ -165,12 +151,24 @@ func (s *server) prepare(ctx context.Context) error { }, MinConnectTimeout: 3 * time.Second, }), - grpc.WithKeepaliveParams(keepalive.ClientParameters{ - Time: 10 * time.Second, - Timeout: 20 * time.Second, - }), - }, - }) + )) + if err != nil { + return errors.Trace(err) + } + s.pdAPIClient, err = pdutil.NewPDAPIClient(s.pdClient, conf.Security) + if err != nil { + return errors.Trace(err) + } + log.Info("create etcdCli", zap.Strings("endpoints", s.pdEndpoints)) + // we do not pass a `context` to create a the etcd client, + // to prevent it's cancelled when the server is closing. + // For example, when the non-owner node goes offline, + // it would resign the campaign key which was put by call `campaign`, + // if this is not done due to the passed context cancelled, + // the key will be kept for the lease TTL, which is 10 seconds, + // then cause the new owner cannot be elected immediately after the old owner offline. + // see https://github.com/etcd-io/etcd/blob/525d53bd41/client/v3/concurrency/election.go#L98 + etcdCli, err := etcd.CreateRawEtcdClient(tlsConfig, grpcTLSOption, s.pdEndpoints...) if err != nil { return errors.Trace(err) } @@ -181,6 +179,15 @@ func (s *server) prepare(ctx context.Context) error { } s.etcdClient = cdcEtcdClient + // Collect all endpoints from pd here to make the server more robust. + // Because in some scenarios, the deployer may only provide one pd endpoint, + // this will cause the TiCDC server to fail to restart when some pd node is down. + allPDEndpoints, err := s.pdAPIClient.CollectMemberEndpoints(ctx) + if err != nil { + return errors.Trace(err) + } + s.pdEndpoints = append(s.pdEndpoints, allPDEndpoints...) + err = s.initDir(ctx) if err != nil { return errors.Trace(err) @@ -192,9 +199,8 @@ func (s *server) prepare(ctx context.Context) error { return errors.Trace(err) } - s.capture = capture.NewCapture( - s.pdEndpoints, cdcEtcdClient, s.grpcService, s.sortEngineFactory) - + s.capture = capture.NewCapture(s.pdEndpoints, cdcEtcdClient, + s.grpcService, s.sortEngineFactory, s.pdClient) return nil } @@ -289,18 +295,7 @@ func (s *server) startStatusHTTP(lis net.Listener) error { return nil } -func (s *server) etcdHealthChecker(ctx context.Context) error { - conf := config.GetGlobalServerConfig() - grpcClient, err := pd.NewClientWithContext(ctx, s.pdEndpoints, conf.Security.PDSecurityOption()) - if err != nil { - return errors.Trace(err) - } - pc, err := pdutil.NewPDAPIClient(grpcClient, conf.Security) - if err != nil { - return errors.Trace(err) - } - defer pc.Close() - +func (s *server) upstreamPDHealthChecker(ctx context.Context) error { ticker := time.NewTicker(time.Second * 3) defer ticker.Stop() @@ -309,7 +304,7 @@ func (s *server) etcdHealthChecker(ctx context.Context) error { case <-ctx.Done(): return ctx.Err() case <-ticker.C: - endpoints, err := pc.CollectMemberEndpoints(ctx) + endpoints, err := s.pdAPIClient.CollectMemberEndpoints(ctx) if err != nil { log.Warn("etcd health check: cannot collect all members", zap.Error(err)) continue @@ -317,7 +312,7 @@ func (s *server) etcdHealthChecker(ctx context.Context) error { for _, endpoint := range endpoints { start := time.Now() ctx, cancel := context.WithTimeout(ctx, 5*time.Second) - if err := pc.Healthy(ctx, endpoint); err != nil { + if err := s.pdAPIClient.Healthy(ctx, endpoint); err != nil { log.Warn("etcd health check error", zap.String("endpoint", endpoint), zap.Error(err)) } @@ -338,6 +333,7 @@ func (s *server) etcdHealthChecker(ctx context.Context) error { func (s *server) run(ctx context.Context) (err error) { ctx, cancel := context.WithCancel(ctx) defer cancel() + defer s.pdAPIClient.Close() eg, egCtx := errgroup.WithContext(ctx) @@ -346,7 +342,7 @@ func (s *server) run(ctx context.Context) (err error) { }) eg.Go(func() error { - return s.etcdHealthChecker(egCtx) + return s.upstreamPDHealthChecker(egCtx) }) eg.Go(func() error { @@ -375,6 +371,11 @@ func (s *server) run(ctx context.Context) (err error) { // Drain removes tables in the current TiCDC instance. // It's part of graceful shutdown, should be called before Close. func (s *server) Drain() <-chan struct{} { + if s.capture == nil { + done := make(chan struct{}) + close(done) + return done + } return s.capture.Drain() } @@ -401,6 +402,10 @@ func (s *server) Close() { } s.tcpServer = nil } + + if s.pdClient != nil { + s.pdClient.Close() + } } func (s *server) closeSortEngineFactory() { diff --git a/cdc/sink/ddlsink/factory/factory.go b/cdc/sink/ddlsink/factory/factory.go index 1a8107c1bf1..17dc2e8a504 100644 --- a/cdc/sink/ddlsink/factory/factory.go +++ b/cdc/sink/ddlsink/factory/factory.go @@ -16,7 +16,6 @@ package factory import ( "context" "net/url" - "strings" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink/ddlsink" @@ -46,7 +45,7 @@ func New( if err != nil { return nil, cerror.WrapError(cerror.ErrSinkURIInvalid, err) } - scheme := strings.ToLower(sinkURI.Scheme) + scheme := sink.GetScheme(sinkURI) switch scheme { case sink.KafkaScheme, sink.KafkaSSLScheme: factoryCreator := kafka.NewSaramaFactory diff --git a/cdc/sink/ddlsink/mq/ddlproducer/pulsar_ddl_mock_producer.go b/cdc/sink/ddlsink/mq/ddlproducer/pulsar_ddl_mock_producer.go index 4005834c416..561d5515a5c 100644 --- a/cdc/sink/ddlsink/mq/ddlproducer/pulsar_ddl_mock_producer.go +++ b/cdc/sink/ddlsink/mq/ddlproducer/pulsar_ddl_mock_producer.go @@ -17,7 +17,6 @@ import ( "context" "github.com/apache/pulsar-client-go/pulsar" - "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/sink/codec/common" @@ -35,9 +34,6 @@ type PulsarMockProducers struct { func (p *PulsarMockProducers) SyncBroadcastMessage(ctx context.Context, topic string, totalPartitionsNum int32, message *common.Message, ) error { - // call SyncSendMessage - - log.Info("pulsarProducers SyncBroadcastMessage in") return p.SyncSendMessage(ctx, topic, totalPartitionsNum, message) } diff --git a/cdc/sink/ddlsink/mq/ddlproducer/pulsar_ddl_producer.go b/cdc/sink/ddlsink/mq/ddlproducer/pulsar_ddl_producer.go index d360202ba04..19aefcd8dd1 100644 --- a/cdc/sink/ddlsink/mq/ddlproducer/pulsar_ddl_producer.go +++ b/cdc/sink/ddlsink/mq/ddlproducer/pulsar_ddl_producer.go @@ -44,6 +44,7 @@ type pulsarProducers struct { } // SyncBroadcastMessage pulsar consume all partitions +// totalPartitionsNum is not used func (p *pulsarProducers) SyncBroadcastMessage(ctx context.Context, topic string, totalPartitionsNum int32, message *common.Message, ) error { @@ -53,7 +54,7 @@ func (p *pulsarProducers) SyncBroadcastMessage(ctx context.Context, topic string } // SyncSendMessage sends a message -// partitionNum is not used,pulsar consume all partitions +// partitionNum is not used, pulsar consume all partitions func (p *pulsarProducers) SyncSendMessage(ctx context.Context, topic string, partitionNum int32, message *common.Message, ) error { @@ -77,6 +78,12 @@ func (p *pulsarProducers) SyncSendMessage(ctx context.Context, topic string, return err } + if message.Type == model.MessageTypeDDL { + log.Info("pulsarProducers SyncSendMessage success", + zap.Any("mID", mID), zap.String("topic", topic), + zap.String("ddl", string(message.Value))) + } + log.Debug("pulsarProducers SyncSendMessage success", zap.Any("mID", mID), zap.String("topic", topic)) @@ -96,7 +103,7 @@ func NewPulsarProducer( zap.String("namespace", changefeedID.Namespace), zap.String("changefeed", changefeedID.ID)) - topicName, err := util.GetTopic(pConfig.GetSinkURI()) + topicName, err := util.GetTopic(pConfig.SinkURI) if err != nil { return nil, err } diff --git a/cdc/sink/ddlsink/mq/kafka_ddl_sink.go b/cdc/sink/ddlsink/mq/kafka_ddl_sink.go index d5e6fba688e..f77571db39d 100644 --- a/cdc/sink/ddlsink/mq/kafka_ddl_sink.go +++ b/cdc/sink/ddlsink/mq/kafka_ddl_sink.go @@ -88,7 +88,7 @@ func NewKafkaDDLSink( return nil, errors.Trace(err) } - eventRouter, err := dispatcher.NewEventRouter(replicaConfig, topic, sinkURI.Scheme) + eventRouter, err := dispatcher.NewEventRouter(replicaConfig, protocol, topic, sinkURI.Scheme) if err != nil { return nil, errors.Trace(err) } @@ -104,7 +104,8 @@ func NewKafkaDDLSink( } start := time.Now() - log.Info("Try to create a DDL sink producer", zap.Any("options", options)) + log.Info("Try to create a DDL sink producer", + zap.String("changefeed", changefeedID.String())) syncProducer, err := factory.SyncProducer(ctx) if err != nil { return nil, errors.Trace(err) diff --git a/cdc/sink/ddlsink/mq/mq_ddl_sink.go b/cdc/sink/ddlsink/mq/mq_ddl_sink.go index f93fcab3969..d8167212b8a 100644 --- a/cdc/sink/ddlsink/mq/mq_ddl_sink.go +++ b/cdc/sink/ddlsink/mq/mq_ddl_sink.go @@ -31,6 +31,26 @@ import ( "go.uber.org/zap" ) +// DDLDispatchRule is the dispatch rule for DDL event. +type DDLDispatchRule int + +const ( + // PartitionZero means the DDL event will be dispatched to partition 0. + // NOTICE: Only for canal and canal-json protocol. + PartitionZero DDLDispatchRule = iota + // PartitionAll means the DDL event will be broadcast to all the partitions. + PartitionAll +) + +func getDDLDispatchRule(protocol config.Protocol) DDLDispatchRule { + switch protocol { + case config.ProtocolCanal, config.ProtocolCanalJSON: + return PartitionZero + default: + } + return PartitionAll +} + // Assert Sink implementation var _ ddlsink.Sink = (*DDLSink)(nil) @@ -94,32 +114,29 @@ func (k *DDLSink) WriteDDLEvent(ctx context.Context, ddl *model.DDLEvent) error } topic := k.eventRouter.GetTopicForDDL(ddl) - partitionRule := k.eventRouter.GetDLLDispatchRuleByProtocol(k.protocol) + partitionRule := getDDLDispatchRule(k.protocol) log.Debug("Emit ddl event", zap.Uint64("commitTs", ddl.CommitTs), zap.String("query", ddl.Query), zap.String("namespace", k.id.Namespace), zap.String("changefeed", k.id.ID)) - if partitionRule == dispatcher.PartitionAll { - partitionNum, err := k.topicManager.GetPartitionNum(ctx, topic) - if err != nil { - return errors.Trace(err) - } - err = k.statistics.RecordDDLExecution(func() error { - return k.producer.SyncBroadcastMessage(ctx, topic, partitionNum, msg) - }) - return errors.Trace(err) - } // Notice: We must call GetPartitionNum here, // which will be responsible for automatically creating topics when they don't exist. // If it is not called here and kafka has `auto.create.topics.enable` turned on, // then the auto-created topic will not be created as configured by ticdc. - _, err = k.topicManager.GetPartitionNum(ctx, topic) + partitionNum, err := k.topicManager.GetPartitionNum(ctx, topic) if err != nil { return errors.Trace(err) } + + if partitionRule == PartitionAll { + err = k.statistics.RecordDDLExecution(func() error { + return k.producer.SyncBroadcastMessage(ctx, topic, partitionNum, msg) + }) + return errors.Trace(err) + } err = k.statistics.RecordDDLExecution(func() error { - return k.producer.SyncSendMessage(ctx, topic, dispatcher.PartitionZero, msg) + return k.producer.SyncSendMessage(ctx, topic, 0, msg) }) return errors.Trace(err) } diff --git a/cdc/sink/ddlsink/mq/mq_ddl_sink_test.go b/cdc/sink/ddlsink/mq/mq_ddl_sink_test.go index 7d3c47c2b38..4403128cdd9 100644 --- a/cdc/sink/ddlsink/mq/mq_ddl_sink_test.go +++ b/cdc/sink/ddlsink/mq/mq_ddl_sink_test.go @@ -279,3 +279,16 @@ func TestWriteCheckpointTsWhenCanalJsonTiDBExtensionIsDisable(t *testing.T) { require.Len(t, s.producer.(*ddlproducer.MockDDLProducer).GetAllEvents(), 0, "No topic and partition should be broadcast") } + +func TestGetDLLDispatchRuleByProtocol(t *testing.T) { + t.Parallel() + + require.Equal(t, PartitionZero, getDDLDispatchRule(config.ProtocolCanal)) + require.Equal(t, PartitionZero, getDDLDispatchRule(config.ProtocolCanalJSON)) + + require.Equal(t, PartitionAll, getDDLDispatchRule(config.ProtocolOpen)) + require.Equal(t, PartitionAll, getDDLDispatchRule(config.ProtocolDefault)) + require.Equal(t, PartitionAll, getDDLDispatchRule(config.ProtocolAvro)) + require.Equal(t, PartitionAll, getDDLDispatchRule(config.ProtocolMaxwell)) + require.Equal(t, PartitionAll, getDDLDispatchRule(config.ProtocolCraft)) +} diff --git a/cdc/sink/ddlsink/mq/pulsar_ddl_sink.go b/cdc/sink/ddlsink/mq/pulsar_ddl_sink.go index a19ccc754a9..4cebead0acc 100644 --- a/cdc/sink/ddlsink/mq/pulsar_ddl_sink.go +++ b/cdc/sink/ddlsink/mq/pulsar_ddl_sink.go @@ -62,10 +62,11 @@ func NewPulsarDDLSink( return nil, errors.Trace(err) } - log.Info("Try to create a DDL sink producer", zap.Any("pulsarConfig", pConfig)) + log.Info("Try to create a DDL sink producer", + zap.String("changefeed", changefeedID.String())) // NewEventRouter - eventRouter, err := dispatcher.NewEventRouter(replicaConfig, defaultTopic, sinkURI.Scheme) + eventRouter, err := dispatcher.NewEventRouter(replicaConfig, protocol, defaultTopic, sinkURI.Scheme) if err != nil { return nil, errors.Trace(err) } diff --git a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go index cddcc50a2f0..e0bc7d9659d 100644 --- a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go +++ b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go @@ -84,6 +84,11 @@ func NewDDLSink( return nil, err } + cfg.IsTiDB, err = pmysql.CheckIsTiDB(ctx, db) + if err != nil { + return nil, err + } + m := &DDLSink{ id: changefeedID, db: db, @@ -99,6 +104,9 @@ func NewDDLSink( // WriteDDLEvent writes a DDL event to the mysql database. func (m *DDLSink) WriteDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { + if ddl.Type == timodel.ActionAddIndex && m.cfg.IsTiDB { + return m.asyncExecAddIndexDDLIfTimeout(ctx, ddl) + } return m.execDDLWithMaxRetries(ctx, ddl) } @@ -247,3 +255,55 @@ func (m *DDLSink) Close() { } } } + +// asyncExecAddIndexDDLIfTimeout executes ddl in async mode. +// this function only works in TiDB, because TiDB will save ddl jobs +// and execute them asynchronously even if ticdc crashed. +func (m *DDLSink) asyncExecAddIndexDDLIfTimeout(ctx context.Context, ddl *model.DDLEvent) error { + done := make(chan error, 1) + // wait for 2 seconds at most + tick := time.NewTimer(2 * time.Second) + defer tick.Stop() + log.Info("async exec add index ddl start", + zap.String("changefeedID", m.id.String()), + zap.Uint64("commitTs", ddl.CommitTs), + zap.String("ddl", ddl.Query)) + go func() { + if err := m.execDDLWithMaxRetries(ctx, ddl); err != nil { + log.Error("async exec add index ddl failed", + zap.String("changefeedID", m.id.String()), + zap.Uint64("commitTs", ddl.CommitTs), + zap.String("ddl", ddl.Query)) + done <- err + return + } + log.Info("async exec add index ddl done", + zap.String("changefeedID", m.id.String()), + zap.Uint64("commitTs", ddl.CommitTs), + zap.String("ddl", ddl.Query)) + done <- nil + }() + + select { + case <-ctx.Done(): + // if the ddl is canceled, we just return nil, if the ddl is not received by tidb, + // the downstream ddl is lost, because the checkpoint ts is forwarded. + log.Info("async add index ddl exits as canceled", + zap.String("changefeedID", m.id.String()), + zap.Uint64("commitTs", ddl.CommitTs), + zap.String("ddl", ddl.Query)) + return nil + case err := <-done: + // if the ddl is executed within 2 seconds, we just return the result to the caller. + return err + case <-tick.C: + // if the ddl is still running, we just return nil, + // then if the ddl is failed, the downstream ddl is lost. + // because the checkpoint ts is forwarded. + log.Info("async add index ddl is still running", + zap.String("changefeedID", m.id.String()), + zap.Uint64("commitTs", ddl.CommitTs), + zap.String("ddl", ddl.Query)) + return nil + } +} diff --git a/cdc/sink/ddlsink/mysql/mysql_ddl_sink_test.go b/cdc/sink/ddlsink/mysql/mysql_ddl_sink_test.go index 1d478f21e85..6f892663fa3 100644 --- a/cdc/sink/ddlsink/mysql/mysql_ddl_sink_test.go +++ b/cdc/sink/ddlsink/mysql/mysql_ddl_sink_test.go @@ -17,7 +17,9 @@ import ( "context" "database/sql" "net/url" + "sync/atomic" "testing" + "time" "github.com/DATA-DOG/go-sqlmock" dmysql "github.com/go-sql-driver/mysql" @@ -30,8 +32,6 @@ import ( ) func TestWriteDDLEvent(t *testing.T) { - t.Parallel() - dbIndex := 0 GetDBConnImpl = func(ctx context.Context, dsnStr string) (*sql.DB, error) { defer func() { @@ -46,6 +46,8 @@ func TestWriteDDLEvent(t *testing.T) { // normal db db, mock, err := sqlmock.New(sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual)) require.Nil(t, err) + mock.ExpectQuery("select tidb_version()"). + WillReturnRows(sqlmock.NewRows([]string{"tidb_version()"}).AddRow("5.7.25-TiDB-v4.0.0-beta-191-ga1b3e3b")) mock.ExpectBegin() mock.ExpectExec("USE `test`;").WillReturnResult(sqlmock.NewResult(1, 1)) mock.ExpectExec("ALTER TABLE test.t1 ADD COLUMN a int").WillReturnResult(sqlmock.NewResult(1, 1)) @@ -142,3 +144,62 @@ func TestNeedSwitchDB(t *testing.T) { require.Equal(t, tc.needSwitch, needSwitchDB(tc.ddl)) } } + +func TestAsyncExecAddIndex(t *testing.T) { + ddlExecutionTime := time.Millisecond * 3000 + var dbIndex int32 = 0 + GetDBConnImpl = func(ctx context.Context, dsnStr string) (*sql.DB, error) { + defer func() { + atomic.AddInt32(&dbIndex, 1) + }() + if atomic.LoadInt32(&dbIndex) == 0 { + // test db + db, err := pmysql.MockTestDB(true) + require.Nil(t, err) + return db, nil + } + // normal db + db, mock, err := sqlmock.New(sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual)) + require.Nil(t, err) + mock.ExpectQuery("select tidb_version()"). + WillReturnRows(sqlmock.NewRows([]string{"tidb_version()"}).AddRow("5.7.25-TiDB-v4.0.0-beta-191-ga1b3e3b")) + mock.ExpectBegin() + mock.ExpectExec("USE `test`;"). + WillReturnResult(sqlmock.NewResult(1, 1)) + mock.ExpectExec("Create index idx1 on test.t1(a)"). + WillDelayFor(ddlExecutionTime). + WillReturnResult(sqlmock.NewResult(1, 1)) + mock.ExpectCommit() + mock.ExpectClose() + return db, nil + } + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + changefeed := "test-changefeed" + sinkURI, err := url.Parse("mysql://127.0.0.1:4000") + require.Nil(t, err) + rc := config.GetDefaultReplicaConfig() + sink, err := NewDDLSink(ctx, model.DefaultChangeFeedID(changefeed), sinkURI, rc) + + require.Nil(t, err) + + ddl1 := &model.DDLEvent{ + StartTs: 1000, + CommitTs: 1010, + TableInfo: &model.TableInfo{ + TableName: model.TableName{ + Schema: "test", + Table: "t1", + }, + }, + Type: timodel.ActionAddIndex, + Query: "Create index idx1 on test.t1(a)", + } + start := time.Now() + err = sink.WriteDDLEvent(ctx, ddl1) + require.Nil(t, err) + require.True(t, time.Since(start) < ddlExecutionTime) + require.True(t, time.Since(start) >= 2*time.Second) + sink.Close() +} diff --git a/cdc/sink/dmlsink/blackhole/black_hole_dml_sink.go b/cdc/sink/dmlsink/blackhole/black_hole_dml_sink.go index 673a2ff2b97..6cc1d1f35dd 100644 --- a/cdc/sink/dmlsink/blackhole/black_hole_dml_sink.go +++ b/cdc/sink/dmlsink/blackhole/black_hole_dml_sink.go @@ -17,6 +17,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink/dmlsink" + "github.com/pingcap/tiflow/pkg/sink" "go.uber.org/zap" ) @@ -42,6 +43,11 @@ func (s *DMLSink) WriteEvents(rows ...*dmlsink.CallbackableEvent[*model.RowChang return nil } +// Scheme return the scheme of the sink. +func (s *DMLSink) Scheme() string { + return sink.BlackHoleScheme +} + // Close do nothing. func (s *DMLSink) Close() {} diff --git a/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink.go b/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink.go index 6044fc52596..6d13ba5359f 100644 --- a/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink.go +++ b/cdc/sink/dmlsink/cloudstorage/cloud_storage_dml_sink.go @@ -17,6 +17,7 @@ import ( "context" "math" "net/url" + "strings" "sync" "sync/atomic" @@ -65,6 +66,7 @@ type eventFragment struct { // It will send the events to cloud storage systems. type DMLSink struct { changefeedID model.ChangeFeedID + scheme string // last sequence number lastSeqNum uint64 // encodingWorkers defines a group of workers for encoding events. @@ -133,6 +135,7 @@ func NewDMLSink(ctx context.Context, wgCtx, wgCancel := context.WithCancel(ctx) s := &DMLSink{ changefeedID: changefeedID, + scheme: strings.ToLower(sinkURI.Scheme), encodingWorkers: make([]*encodingWorker, defaultEncodingConcurrency), workers: make([]*dmlWorker, cfg.WorkerCount), statistics: metrics.NewStatistics(wgCtx, changefeedID, sink.TxnSink), @@ -267,3 +270,8 @@ func (s *DMLSink) Close() { func (s *DMLSink) Dead() <-chan struct{} { return s.dead } + +// Scheme returns the sink scheme. +func (s *DMLSink) Scheme() string { + return s.scheme +} diff --git a/cdc/sink/dmlsink/event.go b/cdc/sink/dmlsink/event.go index da9466a90de..8df85cab249 100644 --- a/cdc/sink/dmlsink/event.go +++ b/cdc/sink/dmlsink/event.go @@ -23,7 +23,7 @@ type TableEvent interface { // GetCommitTs returns the commit timestamp of the event. GetCommitTs() uint64 // TrySplitAndSortUpdateEvent split the update to delete and insert if the unique key is updated - TrySplitAndSortUpdateEvent() error + TrySplitAndSortUpdateEvent(scheme string) error } // CallbackFunc is the callback function for callbackable event. diff --git a/cdc/sink/dmlsink/event_sink.go b/cdc/sink/dmlsink/event_sink.go index 0de7e79f060..76a179ecc9c 100644 --- a/cdc/sink/dmlsink/event_sink.go +++ b/cdc/sink/dmlsink/event_sink.go @@ -18,6 +18,10 @@ type EventSink[E TableEvent] interface { // WriteEvents writes events to the sink. // This is an asynchronously and thread-safe method. WriteEvents(events ...*CallbackableEvent[E]) error + + // Scheme returns the sink scheme. + Scheme() string + // Close closes the sink. Can be called with `WriteEvents` concurrently. Close() // The EventSink meets internal errors and has been dead already. diff --git a/cdc/sink/dmlsink/factory/factory.go b/cdc/sink/dmlsink/factory/factory.go index 5b666803403..5d520630fe9 100644 --- a/cdc/sink/dmlsink/factory/factory.go +++ b/cdc/sink/dmlsink/factory/factory.go @@ -16,7 +16,6 @@ package factory import ( "context" "net/url" - "strings" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" @@ -39,14 +38,29 @@ import ( "github.com/prometheus/client_golang/prometheus" ) +// Category is for different DML sink categories. +type Category = int + +const ( + // CategoryTxn is for Txn sink. + CategoryTxn Category = 1 + // CategoryMQ is for MQ sink. + CategoryMQ = 2 + // CategoryCloudStorage is for CloudStorage sink. + CategoryCloudStorage = 3 + // CategoryBlackhole is for Blackhole sink. + CategoryBlackhole = 4 +) + // SinkFactory is the factory of sink. // It is responsible for creating sink and closing it. // Because there is no way to convert the eventsink.EventSink[*model.RowChangedEvent] // to eventsink.EventSink[eventsink.TableEvent]. // So we have to use this factory to create and store the sink. type SinkFactory struct { - rowSink dmlsink.EventSink[*model.RowChangedEvent] - txnSink dmlsink.EventSink[*model.SingleTableTxn] + rowSink dmlsink.EventSink[*model.RowChangedEvent] + txnSink dmlsink.EventSink[*model.SingleTableTxn] + category Category } // New creates a new SinkFactory by schema. @@ -63,7 +77,7 @@ func New( } s := &SinkFactory{} - schema := strings.ToLower(sinkURI.Scheme) + schema := sink.GetScheme(sinkURI) switch schema { case sink.MySQLScheme, sink.MySQLSSLScheme, sink.TiDBScheme, sink.TiDBSSLScheme: txnSink, err := txn.NewMySQLSink(ctx, changefeedID, sinkURI, cfg, errCh, @@ -72,6 +86,7 @@ func New( return nil, err } s.txnSink = txnSink + s.category = CategoryTxn case sink.KafkaScheme, sink.KafkaSSLScheme: factoryCreator := kafka.NewSaramaFactory if util.GetOrZero(cfg.Sink.EnableKafkaSinkV2) { @@ -83,15 +98,18 @@ func New( return nil, err } s.txnSink = mqs + s.category = CategoryMQ case sink.S3Scheme, sink.FileScheme, sink.GCSScheme, sink.GSScheme, sink.AzblobScheme, sink.AzureScheme, sink.CloudStorageNoopScheme: storageSink, err := cloudstorage.NewDMLSink(ctx, changefeedID, sinkURI, cfg, errCh) if err != nil { return nil, err } s.txnSink = storageSink + s.category = CategoryCloudStorage case sink.BlackHoleScheme: bs := blackhole.NewDMLSink() s.rowSink = bs + s.category = CategoryBlackhole case sink.PulsarScheme: mqs, err := mq.NewPulsarDMLSink(ctx, changefeedID, sinkURI, cfg, errCh, manager.NewPulsarTopicManager, @@ -100,6 +118,7 @@ func New( return nil, err } s.txnSink = mqs + s.category = CategoryMQ default: return nil, cerror.ErrSinkURIInvalid.GenWithStack("the sink scheme (%s) is not supported", schema) @@ -156,3 +175,11 @@ func (s *SinkFactory) Close() { s.txnSink.Close() } } + +// Category returns category of s. +func (s *SinkFactory) Category() Category { + if s.category == 0 { + panic("should never happen") + } + return s.category +} diff --git a/cdc/sink/dmlsink/mq/dispatcher/event_router.go b/cdc/sink/dmlsink/mq/dispatcher/event_router.go index 9d1f604a979..37731327c54 100644 --- a/cdc/sink/dmlsink/mq/dispatcher/event_router.go +++ b/cdc/sink/dmlsink/mq/dispatcher/event_router.go @@ -24,55 +24,15 @@ import ( "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/sink" - "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" ) -// DDLDispatchRule is the dispatch rule for DDL event. -type DDLDispatchRule int - -const ( - // PartitionAll means the DDL event will be broadcast to all the partitions. - PartitionAll DDLDispatchRule = -1 - // PartitionZero means the DDL event will be dispatched to partition 0. - // NOTICE: Only for canal and canal-json protocol. - PartitionZero = 0 -) - -type partitionDispatchRule int - -const ( - partitionDispatchRuleDefault partitionDispatchRule = iota - partitionDispatchRuleTS - partitionDispatchRuleTable - partitionDispatchRuleIndexValue -) - -func (r *partitionDispatchRule) fromString(rule string) { - switch strings.ToLower(rule) { - case "default": - *r = partitionDispatchRuleDefault - case "ts": - *r = partitionDispatchRuleTS - case "table": - *r = partitionDispatchRuleTable - case "rowid": - *r = partitionDispatchRuleIndexValue - log.Warn("rowid is deprecated, please use index-value instead.") - case "index-value": - *r = partitionDispatchRuleIndexValue - default: - *r = partitionDispatchRuleDefault - log.Warn("the partition dispatch rule is not default/ts/table/index-value," + - " use the default rule instead.") - } -} - // EventRouter is a router, it determines which topic and which partition // an event should be dispatched to. type EventRouter struct { defaultTopic string - rules []struct { + + rules []struct { partitionDispatcher partition.Dispatcher topicDispatcher topic.Dispatcher filter.Filter @@ -80,7 +40,9 @@ type EventRouter struct { } // NewEventRouter creates a new EventRouter. -func NewEventRouter(cfg *config.ReplicaConfig, defaultTopic, schema string) (*EventRouter, error) { +func NewEventRouter( + cfg *config.ReplicaConfig, protocol config.Protocol, defaultTopic, scheme string, +) (*EventRouter, error) { // If an event does not match any dispatching rules in the config file, // it will be dispatched by the default partition dispatcher and // static topic dispatcher because it matches *.* rule. @@ -89,6 +51,7 @@ func NewEventRouter(cfg *config.ReplicaConfig, defaultTopic, schema string) (*Ev PartitionRule: "default", TopicRule: "", }) + rules := make([]struct { partitionDispatcher partition.Dispatcher topicDispatcher topic.Dispatcher @@ -104,9 +67,8 @@ func NewEventRouter(cfg *config.ReplicaConfig, defaultTopic, schema string) (*Ev f = filter.CaseInsensitive(f) } - d := getPartitionDispatcher(ruleConfig, cfg.EnableOldValue) - t, err := getTopicDispatcher(ruleConfig, defaultTopic, - util.GetOrZero(cfg.Sink.Protocol), schema) + d := getPartitionDispatcher(ruleConfig.PartitionRule, scheme) + t, err := getTopicDispatcher(ruleConfig.TopicRule, defaultTopic, protocol, scheme) if err != nil { return nil, err } @@ -154,7 +116,7 @@ func (s *EventRouter) GetTopicForDDL(ddl *model.DDLEvent) string { func (s *EventRouter) GetPartitionForRowChange( row *model.RowChangedEvent, partitionNum int32, -) int32 { +) (int32, string) { _, partitionDispatcher := s.matchDispatcher( row.Table.Schema, row.Table.Table, ) @@ -164,17 +126,6 @@ func (s *EventRouter) GetPartitionForRowChange( ) } -// GetDLLDispatchRuleByProtocol returns the DDL -// distribution rule according to the protocol. -func (s *EventRouter) GetDLLDispatchRuleByProtocol( - protocol config.Protocol, -) DDLDispatchRule { - if protocol == config.ProtocolCanal || protocol == config.ProtocolCanalJSON { - return PartitionZero - } - return PartitionAll -} - // GetActiveTopics returns a list of the corresponding topics // for the tables that are actively synchronized. func (s *EventRouter) GetActiveTopics(activeTables []model.TableName) []string { @@ -225,68 +176,56 @@ func (s *EventRouter) matchDispatcher( } // getPartitionDispatcher returns the partition dispatcher for a specific partition rule. -func getPartitionDispatcher( - ruleConfig *config.DispatchRule, enableOldValue bool, -) partition.Dispatcher { - var ( - d partition.Dispatcher - rule partitionDispatchRule - ) - rule.fromString(ruleConfig.PartitionRule) - switch rule { - case partitionDispatchRuleIndexValue: - if enableOldValue { - log.Warn("This index-value distribution mode " + - "does not guarantee row-level orderliness when " + - "switching on the old value, so please use caution!") - } - d = partition.NewIndexValueDispatcher() - case partitionDispatchRuleTS: - d = partition.NewTsDispatcher() - case partitionDispatchRuleTable: - d = partition.NewTableDispatcher() - case partitionDispatchRuleDefault: - d = partition.NewDefaultDispatcher(enableOldValue) +func getPartitionDispatcher(rule string, scheme string) partition.Dispatcher { + switch strings.ToLower(rule) { + case "default": + return partition.NewDefaultDispatcher() + case "ts": + return partition.NewTsDispatcher() + case "table": + return partition.NewTableDispatcher() + case "index-value", "rowid": + log.Warn("rowid is deprecated, please use index-value instead.") + return partition.NewIndexValueDispatcher() + default: + } + + if sink.IsPulsarScheme(scheme) { + return partition.NewKeyDispatcher(rule) } - return d + log.Warn("the partition dispatch rule is not default/ts/table/index-value," + + " use the default rule instead.") + return partition.NewDefaultDispatcher() } // getTopicDispatcher returns the topic dispatcher for a specific topic rule (aka topic expression). func getTopicDispatcher( - ruleConfig *config.DispatchRule, defaultTopic, protocol, schema string, + rule string, defaultTopic string, protocol config.Protocol, schema string, ) (topic.Dispatcher, error) { - if ruleConfig.TopicRule == "" { + if rule == "" { return topic.NewStaticTopicDispatcher(defaultTopic), nil } // check if this rule is a valid topic expression - topicExpr := topic.Expression(ruleConfig.TopicRule) + topicExpr := topic.Expression(rule) - if protocol != "" { - p, err := config.ParseSinkProtocolFromString(protocol) - if err != nil { - return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, err) - } - - if schema == sink.PulsarScheme { - err = topicExpr.PulsarValidate() - if err != nil { - return nil, err - } - } else { - if p == config.ProtocolAvro { - err = topicExpr.ValidateForAvro() - if err != nil { - return nil, err - } - } else { - err = topicExpr.Validate() - if err != nil { - return nil, err - } - } + var err error + // validate the topic expression for pulsar sink + if sink.IsPulsarScheme(schema) { + err = topicExpr.PulsarValidate() + } else { + // validate the topic expression for kafka sink + switch protocol { + case config.ProtocolAvro: + err = topicExpr.ValidateForAvro() + default: + err = topicExpr.Validate() } } + if err != nil { + return nil, err + } + return topic.NewDynamicTopicDispatcher(topicExpr), nil } diff --git a/cdc/sink/dmlsink/mq/dispatcher/event_router_test.go b/cdc/sink/dmlsink/mq/dispatcher/event_router_test.go index 8042c26dba8..5227f8d47b4 100644 --- a/cdc/sink/dmlsink/mq/dispatcher/event_router_test.go +++ b/cdc/sink/dmlsink/mq/dispatcher/event_router_test.go @@ -21,45 +21,43 @@ import ( "github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/dispatcher/partition" "github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/dispatcher/topic" "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/sink" "github.com/stretchr/testify/require" ) -func TestEventRouter(t *testing.T) { - t.Parallel() - - d, err := NewEventRouter(config.GetDefaultReplicaConfig(), "test", "kafka") - require.Nil(t, err) - require.Equal(t, "test", d.GetDefaultTopic()) - topicDispatcher, partitionDispatcher := d.matchDispatcher("test", "test") - require.IsType(t, &topic.StaticTopicDispatcher{}, topicDispatcher) - require.IsType(t, &partition.DefaultDispatcher{}, partitionDispatcher) - - d, err = NewEventRouter(&config.ReplicaConfig{ +func newReplicaConfig4DispatcherTest() *config.ReplicaConfig { + return &config.ReplicaConfig{ Sink: &config.SinkConfig{ DispatchRules: []*config.DispatchRule{ + // rule-0 { Matcher: []string{"test_default1.*"}, PartitionRule: "default", }, + // rule-1 { Matcher: []string{"test_default2.*"}, PartitionRule: "unknown-dispatcher", }, + // rule-2 { Matcher: []string{"test_table.*"}, PartitionRule: "table", TopicRule: "hello_{schema}_world", }, + // rule-3 { Matcher: []string{"test_index_value.*"}, PartitionRule: "index-value", TopicRule: "{schema}_world", }, + // rule-4 { Matcher: []string{"test.*"}, PartitionRule: "rowid", TopicRule: "hello_{schema}", }, + // rule-5 { Matcher: []string{"*.*", "!*.test"}, PartitionRule: "ts", @@ -67,75 +65,69 @@ func TestEventRouter(t *testing.T) { }, }, }, - }, "", "kafka") - require.Nil(t, err) - topicDispatcher, partitionDispatcher = d.matchDispatcher("test", "table1") - require.IsType(t, &topic.DynamicTopicDispatcher{}, topicDispatcher) - require.IsType(t, &partition.IndexValueDispatcher{}, partitionDispatcher) + } +} - topicDispatcher, partitionDispatcher = d.matchDispatcher("sbs", "table2") - require.IsType(t, &topic.DynamicTopicDispatcher{}, topicDispatcher) - require.IsType(t, &partition.TsDispatcher{}, partitionDispatcher) +func TestEventRouter(t *testing.T) { + t.Parallel() + replicaConfig := config.GetDefaultReplicaConfig() + d, err := NewEventRouter(replicaConfig, config.ProtocolCanalJSON, "test", sink.KafkaScheme) + require.NoError(t, err) + require.Equal(t, "test", d.GetDefaultTopic()) + topicDispatcher, partitionDispatcher := d.matchDispatcher("test", "test") + require.IsType(t, &topic.StaticTopicDispatcher{}, topicDispatcher) + require.IsType(t, &partition.DefaultDispatcher{}, partitionDispatcher) + + actual := topicDispatcher.Substitute("test", "test") + require.Equal(t, d.defaultTopic, actual) + + replicaConfig = newReplicaConfig4DispatcherTest() + d, err = NewEventRouter(replicaConfig, config.ProtocolCanalJSON, "", sink.KafkaScheme) + require.NoError(t, err) + + // no matched, use the default topicDispatcher, partitionDispatcher = d.matchDispatcher("sbs", "test") require.IsType(t, &topic.StaticTopicDispatcher{}, topicDispatcher) require.IsType(t, &partition.DefaultDispatcher{}, partitionDispatcher) + // match rule-0 topicDispatcher, partitionDispatcher = d.matchDispatcher("test_default1", "test") require.IsType(t, &topic.StaticTopicDispatcher{}, topicDispatcher) require.IsType(t, &partition.DefaultDispatcher{}, partitionDispatcher) + // match rule-1 topicDispatcher, partitionDispatcher = d.matchDispatcher("test_default2", "test") require.IsType(t, &topic.StaticTopicDispatcher{}, topicDispatcher) require.IsType(t, &partition.DefaultDispatcher{}, partitionDispatcher) + // match rule-2 topicDispatcher, partitionDispatcher = d.matchDispatcher("test_table", "test") require.IsType(t, &topic.DynamicTopicDispatcher{}, topicDispatcher) require.IsType(t, &partition.TableDispatcher{}, partitionDispatcher) + // match rule-4 topicDispatcher, partitionDispatcher = d.matchDispatcher("test_index_value", "test") require.IsType(t, &topic.DynamicTopicDispatcher{}, topicDispatcher) require.IsType(t, &partition.IndexValueDispatcher{}, partitionDispatcher) + + // match rule-4 + topicDispatcher, partitionDispatcher = d.matchDispatcher("test", "table1") + require.IsType(t, &topic.DynamicTopicDispatcher{}, topicDispatcher) + require.IsType(t, &partition.IndexValueDispatcher{}, partitionDispatcher) + + // match rule-5 + topicDispatcher, partitionDispatcher = d.matchDispatcher("sbs", "table2") + require.IsType(t, &topic.DynamicTopicDispatcher{}, topicDispatcher) + require.IsType(t, &partition.TsDispatcher{}, partitionDispatcher) } func TestGetActiveTopics(t *testing.T) { t.Parallel() - d, err := NewEventRouter(&config.ReplicaConfig{ - Sink: &config.SinkConfig{ - DispatchRules: []*config.DispatchRule{ - { - Matcher: []string{"test_default1.*"}, - PartitionRule: "default", - }, - { - Matcher: []string{"test_default2.*"}, - PartitionRule: "unknown-dispatcher", - }, - { - Matcher: []string{"test_table.*"}, - PartitionRule: "table", - TopicRule: "hello_{schema}_world", - }, - { - Matcher: []string{"test_index_value.*"}, - PartitionRule: "index-value", - TopicRule: "{schema}_world", - }, - { - Matcher: []string{"test.*"}, - PartitionRule: "rowid", - TopicRule: "hello_{schema}", - }, - { - Matcher: []string{"*.*", "!*.test"}, - PartitionRule: "ts", - TopicRule: "{schema}_{table}", - }, - }, - }, - }, "test", "kafka") - require.Nil(t, err) + replicaConfig := newReplicaConfig4DispatcherTest() + d, err := NewEventRouter(replicaConfig, config.ProtocolCanalJSON, "test", sink.KafkaScheme) + require.NoError(t, err) names := []model.TableName{ {Schema: "test_default1", Table: "table"}, {Schema: "test_default2", Table: "table"}, @@ -151,58 +143,30 @@ func TestGetActiveTopics(t *testing.T) { func TestGetTopicForRowChange(t *testing.T) { t.Parallel() - d, err := NewEventRouter(&config.ReplicaConfig{ - Sink: &config.SinkConfig{ - DispatchRules: []*config.DispatchRule{ - { - Matcher: []string{"test_default1.*"}, - PartitionRule: "default", - }, - { - Matcher: []string{"test_default2.*"}, - PartitionRule: "unknown-dispatcher", - }, - { - Matcher: []string{"test_table.*"}, - PartitionRule: "table", - TopicRule: "hello_{schema}_world", - }, - { - Matcher: []string{"test_index_value.*"}, - PartitionRule: "index-value", - TopicRule: "{schema}_world", - }, - { - Matcher: []string{"test.*"}, - PartitionRule: "rowid", - TopicRule: "hello_{schema}", - }, - { - Matcher: []string{"*.*", "!*.test"}, - PartitionRule: "ts", - TopicRule: "{schema}_{table}", - }, - }, - }, - }, "test", "kafka") - require.Nil(t, err) + replicaConfig := newReplicaConfig4DispatcherTest() + d, err := NewEventRouter(replicaConfig, config.ProtocolCanalJSON, "test", "kafka") + require.NoError(t, err) topicName := d.GetTopicForRowChange(&model.RowChangedEvent{ Table: &model.TableName{Schema: "test_default1", Table: "table"}, }) require.Equal(t, "test", topicName) + topicName = d.GetTopicForRowChange(&model.RowChangedEvent{ Table: &model.TableName{Schema: "test_default2", Table: "table"}, }) require.Equal(t, "test", topicName) + topicName = d.GetTopicForRowChange(&model.RowChangedEvent{ Table: &model.TableName{Schema: "test_table", Table: "table"}, }) require.Equal(t, "hello_test_table_world", topicName) + topicName = d.GetTopicForRowChange(&model.RowChangedEvent{ Table: &model.TableName{Schema: "test_index_value", Table: "table"}, }) require.Equal(t, "test_index_value_world", topicName) + topicName = d.GetTopicForRowChange(&model.RowChangedEvent{ Table: &model.TableName{Schema: "a", Table: "table"}, }) @@ -212,43 +176,11 @@ func TestGetTopicForRowChange(t *testing.T) { func TestGetPartitionForRowChange(t *testing.T) { t.Parallel() - d, err := NewEventRouter(&config.ReplicaConfig{ - Sink: &config.SinkConfig{ - DispatchRules: []*config.DispatchRule{ - { - Matcher: []string{"test_default1.*"}, - PartitionRule: "default", - }, - { - Matcher: []string{"test_default2.*"}, - PartitionRule: "unknown-dispatcher", - }, - { - Matcher: []string{"test_table.*"}, - PartitionRule: "table", - TopicRule: "hello_{schema}_world", - }, - { - Matcher: []string{"test_index_value.*"}, - PartitionRule: "index-value", - TopicRule: "{schema}_world", - }, - { - Matcher: []string{"test.*"}, - PartitionRule: "rowid", - TopicRule: "hello_{schema}", - }, - { - Matcher: []string{"*.*", "!*.test"}, - PartitionRule: "ts", - TopicRule: "{schema}_{table}", - }, - }, - }, - }, "test", "kafka") - require.Nil(t, err) + replicaConfig := newReplicaConfig4DispatcherTest() + d, err := NewEventRouter(replicaConfig, config.ProtocolCanalJSON, "test", sink.KafkaScheme) + require.NoError(t, err) - p := d.GetPartitionForRowChange(&model.RowChangedEvent{ + p, _ := d.GetPartitionForRowChange(&model.RowChangedEvent{ Table: &model.TableName{Schema: "test_default1", Table: "table"}, Columns: []*model.Column{ { @@ -259,8 +191,9 @@ func TestGetPartitionForRowChange(t *testing.T) { }, IndexColumns: [][]int{{0}}, }, 16) - require.Equal(t, int32(10), p) - p = d.GetPartitionForRowChange(&model.RowChangedEvent{ + require.Equal(t, int32(14), p) + + p, _ = d.GetPartitionForRowChange(&model.RowChangedEvent{ Table: &model.TableName{Schema: "test_default2", Table: "table"}, Columns: []*model.Column{ { @@ -271,14 +204,15 @@ func TestGetPartitionForRowChange(t *testing.T) { }, IndexColumns: [][]int{{0}}, }, 16) - require.Equal(t, int32(4), p) + require.Equal(t, int32(0), p) - p = d.GetPartitionForRowChange(&model.RowChangedEvent{ + p, _ = d.GetPartitionForRowChange(&model.RowChangedEvent{ Table: &model.TableName{Schema: "test_table", Table: "table"}, CommitTs: 1, }, 16) require.Equal(t, int32(15), p) - p = d.GetPartitionForRowChange(&model.RowChangedEvent{ + + p, _ = d.GetPartitionForRowChange(&model.RowChangedEvent{ Table: &model.TableName{Schema: "test_index_value", Table: "table"}, Columns: []*model.Column{ { @@ -293,73 +227,18 @@ func TestGetPartitionForRowChange(t *testing.T) { }, }, 10) require.Equal(t, int32(1), p) - p = d.GetPartitionForRowChange(&model.RowChangedEvent{ + + p, _ = d.GetPartitionForRowChange(&model.RowChangedEvent{ Table: &model.TableName{Schema: "a", Table: "table"}, CommitTs: 1, }, 2) require.Equal(t, int32(1), p) } -func TestGetDLLDispatchRuleByProtocol(t *testing.T) { - t.Parallel() - - d, err := NewEventRouter(&config.ReplicaConfig{ - Sink: &config.SinkConfig{ - DispatchRules: []*config.DispatchRule{ - { - Matcher: []string{"test_table.*"}, - PartitionRule: "table", - TopicRule: "hello_{schema}_world", - }, - }, - }, - }, "test", "kafka") - require.Nil(t, err) - - tests := []struct { - protocol config.Protocol - expectedRule DDLDispatchRule - }{ - { - protocol: config.ProtocolDefault, - expectedRule: PartitionAll, - }, - { - protocol: config.ProtocolCanal, - expectedRule: PartitionZero, - }, - { - protocol: config.ProtocolAvro, - expectedRule: PartitionAll, - }, - { - protocol: config.ProtocolMaxwell, - expectedRule: PartitionAll, - }, - { - protocol: config.ProtocolCanalJSON, - expectedRule: PartitionZero, - }, - { - protocol: config.ProtocolCraft, - expectedRule: PartitionAll, - }, - { - protocol: config.ProtocolOpen, - expectedRule: PartitionAll, - }, - } - - for _, test := range tests { - rule := d.GetDLLDispatchRuleByProtocol(test.protocol) - require.Equal(t, test.expectedRule, rule) - } -} - func TestGetTopicForDDL(t *testing.T) { t.Parallel() - d, err := NewEventRouter(&config.ReplicaConfig{ + replicaConfig := &config.ReplicaConfig{ Sink: &config.SinkConfig{ DispatchRules: []*config.DispatchRule{ { @@ -374,8 +253,10 @@ func TestGetTopicForDDL(t *testing.T) { }, }, }, - }, "test", "kafka") - require.Nil(t, err) + } + + d, err := NewEventRouter(replicaConfig, config.ProtocolDefault, "test", "kafka") + require.NoError(t, err) tests := []struct { ddl *model.DDLEvent diff --git a/cdc/sink/dmlsink/mq/dispatcher/partition/default.go b/cdc/sink/dmlsink/mq/dispatcher/partition/default.go index 2def38c003e..fb6ceb03689 100644 --- a/cdc/sink/dmlsink/mq/dispatcher/partition/default.go +++ b/cdc/sink/dmlsink/mq/dispatcher/partition/default.go @@ -19,28 +19,18 @@ import ( // DefaultDispatcher is the default partition dispatcher. type DefaultDispatcher struct { - tbd *TableDispatcher - ivd *IndexValueDispatcher - enableOldValue bool + tbd *TableDispatcher } // NewDefaultDispatcher creates a DefaultDispatcher. -func NewDefaultDispatcher(enableOldValue bool) *DefaultDispatcher { +func NewDefaultDispatcher() *DefaultDispatcher { return &DefaultDispatcher{ - tbd: NewTableDispatcher(), - ivd: NewIndexValueDispatcher(), - enableOldValue: enableOldValue, + tbd: NewTableDispatcher(), } } // DispatchRowChangedEvent returns the target partition to which // a row changed event should be dispatched. -func (d *DefaultDispatcher) DispatchRowChangedEvent(row *model.RowChangedEvent, partitionNum int32) int32 { - if d.enableOldValue { - return d.tbd.DispatchRowChangedEvent(row, partitionNum) - } - if len(row.IndexColumns) != 1 { - return d.tbd.DispatchRowChangedEvent(row, partitionNum) - } - return d.ivd.DispatchRowChangedEvent(row, partitionNum) +func (d *DefaultDispatcher) DispatchRowChangedEvent(row *model.RowChangedEvent, partitionNum int32) (int32, string) { + return d.tbd.DispatchRowChangedEvent(row, partitionNum) } diff --git a/cdc/sink/dmlsink/mq/dispatcher/partition/default_test.go b/cdc/sink/dmlsink/mq/dispatcher/partition/default_test.go index 5fa6ad69590..b0319241e10 100644 --- a/cdc/sink/dmlsink/mq/dispatcher/partition/default_test.go +++ b/cdc/sink/dmlsink/mq/dispatcher/partition/default_test.go @@ -23,204 +23,21 @@ import ( func TestDefaultDispatcher(t *testing.T) { t.Parallel() - testCases := []struct { - row *model.RowChangedEvent - expectPartition int32 - }{ - {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t1", - }, - Columns: []*model.Column{ - { - Name: "id", - Value: 1, - Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, - }, - }, - IndexColumns: [][]int{{0}}, - }, expectPartition: 11}, - {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t1", - }, - Columns: []*model.Column{ - { - Name: "id", - Value: 2, - Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, - }, - }, - IndexColumns: [][]int{{0}}, - }, expectPartition: 1}, - {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t1", - }, - Columns: []*model.Column{ - { - Name: "id", - Value: 3, - Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, - }, - }, - IndexColumns: [][]int{{0}}, - }, expectPartition: 7}, - {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t2", - }, - Columns: []*model.Column{ - { - Name: "id", - Value: 1, - Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, - }, { - Name: "a", - Value: 1, - }, - }, - IndexColumns: [][]int{{0}}, - }, expectPartition: 1}, - {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t2", - }, - Columns: []*model.Column{ - { - Name: "id", - Value: 2, - Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, - }, { - Name: "a", - Value: 2, - }, - }, - IndexColumns: [][]int{{0}}, - }, expectPartition: 11}, - {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t2", - }, - Columns: []*model.Column{ - { - Name: "id", - Value: 3, - Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, - }, { - Name: "a", - Value: 3, - }, - }, - IndexColumns: [][]int{{0}}, - }, expectPartition: 13}, - {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t2", - }, - Columns: []*model.Column{ - { - Name: "id", - Value: 3, - Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, - }, { - Name: "a", - Value: 4, - }, - }, - IndexColumns: [][]int{{0}}, - }, expectPartition: 13}, - {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t3", - }, - Columns: []*model.Column{ - { - Name: "id", - Value: 1, - Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, - }, - { - Name: "a", - Value: 2, - Flag: model.UniqueKeyFlag, - }, - }, - IndexColumns: [][]int{{0}, {1}}, - }, expectPartition: 3}, - {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t3", - }, - Columns: []*model.Column{ - { - Name: "id", - Value: 2, - Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, - }, { - Name: "a", - Value: 3, - Flag: model.UniqueKeyFlag, - }, - }, - IndexColumns: [][]int{{0}, {1}}, - }, expectPartition: 3}, - {row: &model.RowChangedEvent{ - Table: &model.TableName{ - Schema: "test", - Table: "t3", - }, - Columns: []*model.Column{ - { - Name: "id", - Value: 3, - Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, - }, { - Name: "a", - Value: 4, - Flag: model.UniqueKeyFlag, - }, - }, - IndexColumns: [][]int{{0}, {1}}, - }, expectPartition: 3}, - } - p := NewDefaultDispatcher(false) - for _, tc := range testCases { - require.Equal(t, tc.expectPartition, p.DispatchRowChangedEvent(tc.row, 16)) - } -} - -func TestDefaultDispatcherWithOldValue(t *testing.T) { - t.Parallel() - row := &model.RowChangedEvent{ Table: &model.TableName{ Schema: "test", - Table: "t3", + Table: "t1", }, Columns: []*model.Column{ { Name: "id", - Value: 2, + Value: 1, Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, - }, { - Name: "a", - Value: 3, - Flag: model.UniqueKeyFlag, }, }, - IndexColumns: [][]int{{0}, {1}}, + IndexColumns: [][]int{{0}}, } - p := NewDefaultDispatcher(true) - require.Equal(t, int32(3), p.DispatchRowChangedEvent(row, 16)) + targetPartition, _ := NewDefaultDispatcher().DispatchRowChangedEvent(row, 3) + require.Equal(t, int32(0), targetPartition) } diff --git a/cdc/sink/dmlsink/mq/dispatcher/partition/dispatcher.go b/cdc/sink/dmlsink/mq/dispatcher/partition/dispatcher.go index 2e0e2995121..c0eb739734f 100644 --- a/cdc/sink/dmlsink/mq/dispatcher/partition/dispatcher.go +++ b/cdc/sink/dmlsink/mq/dispatcher/partition/dispatcher.go @@ -19,7 +19,8 @@ import ( // Dispatcher is an abstraction for dispatching rows into different partitions type Dispatcher interface { - // DispatchRowChangedEvent returns an index of partitions according to RowChangedEvent. + // DispatchRowChangedEvent returns an index of partitions or a partition key + // according to RowChangedEvent. // Concurrency Note: This method is thread-safe. - DispatchRowChangedEvent(row *model.RowChangedEvent, partitionNum int32) int32 + DispatchRowChangedEvent(row *model.RowChangedEvent, partitionNum int32) (int32, string) } diff --git a/cdc/sink/dmlsink/mq/dispatcher/partition/index_value.go b/cdc/sink/dmlsink/mq/dispatcher/partition/index_value.go index 3397a00a238..4476a905466 100644 --- a/cdc/sink/dmlsink/mq/dispatcher/partition/index_value.go +++ b/cdc/sink/dmlsink/mq/dispatcher/partition/index_value.go @@ -14,6 +14,7 @@ package partition import ( + "strconv" "sync" "github.com/pingcap/tiflow/cdc/model" @@ -35,15 +36,12 @@ func NewIndexValueDispatcher() *IndexValueDispatcher { // DispatchRowChangedEvent returns the target partition to which // a row changed event should be dispatched. -func (r *IndexValueDispatcher) DispatchRowChangedEvent(row *model.RowChangedEvent, partitionNum int32) int32 { +func (r *IndexValueDispatcher) DispatchRowChangedEvent(row *model.RowChangedEvent, partitionNum int32) (int32, string) { r.lock.Lock() defer r.lock.Unlock() r.hasher.Reset() r.hasher.Write([]byte(row.Table.Schema), []byte(row.Table.Table)) - // FIXME(leoppro): if the row events includes both pre-cols and cols - // the dispatch logic here is wrong - // distribute partition by rowid or unique column value dispatchCols := row.Columns if len(row.Columns) == 0 { dispatchCols = row.PreColumns @@ -56,5 +54,6 @@ func (r *IndexValueDispatcher) DispatchRowChangedEvent(row *model.RowChangedEven r.hasher.Write([]byte(col.Name), []byte(model.ColumnValueString(col.Value))) } } - return int32(r.hasher.Sum32() % uint32(partitionNum)) + sum32 := r.hasher.Sum32() + return int32(sum32 % uint32(partitionNum)), strconv.FormatInt(int64(sum32), 10) } diff --git a/cdc/sink/dmlsink/mq/dispatcher/partition/index_value_test.go b/cdc/sink/dmlsink/mq/dispatcher/partition/index_value_test.go index c68d46ac757..9d702163405 100644 --- a/cdc/sink/dmlsink/mq/dispatcher/partition/index_value_test.go +++ b/cdc/sink/dmlsink/mq/dispatcher/partition/index_value_test.go @@ -149,6 +149,7 @@ func TestIndexValueDispatcher(t *testing.T) { } p := NewIndexValueDispatcher() for _, tc := range testCases { - require.Equal(t, tc.expectPartition, p.DispatchRowChangedEvent(tc.row, 16)) + index, _ := p.DispatchRowChangedEvent(tc.row, 16) + require.Equal(t, tc.expectPartition, index) } } diff --git a/cdc/sink/dmlsink/mq/dispatcher/partition/key.go b/cdc/sink/dmlsink/mq/dispatcher/partition/key.go new file mode 100644 index 00000000000..6f41371fbf1 --- /dev/null +++ b/cdc/sink/dmlsink/mq/dispatcher/partition/key.go @@ -0,0 +1,37 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package partition + +import ( + "github.com/pingcap/tiflow/cdc/model" +) + +// KeyDispatcher is a partition dispatcher which dispatches events +// using the provided partition key. +type KeyDispatcher struct { + partitionKey string +} + +// NewKeyDispatcher creates a TableDispatcher. +func NewKeyDispatcher(partitionKey string) *KeyDispatcher { + return &KeyDispatcher{ + partitionKey: partitionKey, + } +} + +// DispatchRowChangedEvent returns the target partition to which +// a row changed event should be dispatched. +func (t *KeyDispatcher) DispatchRowChangedEvent(*model.RowChangedEvent, int32) (int32, string) { + return 0, t.partitionKey +} diff --git a/cdc/sink/dmlsink/mq/dispatcher/partition/key_test.go b/cdc/sink/dmlsink/mq/dispatcher/partition/key_test.go new file mode 100644 index 00000000000..37843904795 --- /dev/null +++ b/cdc/sink/dmlsink/mq/dispatcher/partition/key_test.go @@ -0,0 +1,51 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed 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, +// See the License for the specific language governing permissions and +// limitations under the License. +package partition + +import ( + "testing" + + "github.com/pingcap/tiflow/cdc/model" +) + +func TestKeyDispatcher_DispatchRowChangedEvent(t *testing.T) { + tests := []struct { + name string + partitionKey string + rowChangedEvt *model.RowChangedEvent + wantPartition int32 + wantKey string + }{ + { + name: "dispatch to partition 0 with partition key 'foo'", + partitionKey: "foo", + rowChangedEvt: &model.RowChangedEvent{}, + wantPartition: 0, + wantKey: "foo", + }, + // Add more test cases here + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + d := NewKeyDispatcher(tt.partitionKey) + gotPartition, gotKey := d.DispatchRowChangedEvent(tt.rowChangedEvt, 0) + if gotPartition != tt.wantPartition { + t.Errorf("DispatchRowChangedEvent() gotPartition = %v, want %v", gotPartition, tt.wantPartition) + } + if gotKey != tt.wantKey { + t.Errorf("DispatchRowChangedEvent() gotKey = %v, want %v", gotKey, tt.wantKey) + } + }) + } +} diff --git a/cdc/sink/dmlsink/mq/dispatcher/partition/table.go b/cdc/sink/dmlsink/mq/dispatcher/partition/table.go index 62744333e72..a577d7bac5e 100644 --- a/cdc/sink/dmlsink/mq/dispatcher/partition/table.go +++ b/cdc/sink/dmlsink/mq/dispatcher/partition/table.go @@ -36,11 +36,11 @@ func NewTableDispatcher() *TableDispatcher { // DispatchRowChangedEvent returns the target partition to which // a row changed event should be dispatched. -func (t *TableDispatcher) DispatchRowChangedEvent(row *model.RowChangedEvent, partitionNum int32) int32 { +func (t *TableDispatcher) DispatchRowChangedEvent(row *model.RowChangedEvent, partitionNum int32) (int32, string) { t.lock.Lock() defer t.lock.Unlock() t.hasher.Reset() // distribute partition by table t.hasher.Write([]byte(row.Table.Schema), []byte(row.Table.Table)) - return int32(t.hasher.Sum32() % uint32(partitionNum)) + return int32(t.hasher.Sum32() % uint32(partitionNum)), row.Table.String() } diff --git a/cdc/sink/dmlsink/mq/dispatcher/partition/table_test.go b/cdc/sink/dmlsink/mq/dispatcher/partition/table_test.go index 8f1cbb8db4e..468dae23994 100644 --- a/cdc/sink/dmlsink/mq/dispatcher/partition/table_test.go +++ b/cdc/sink/dmlsink/mq/dispatcher/partition/table_test.go @@ -79,6 +79,7 @@ func TestTableDispatcher(t *testing.T) { } p := NewTableDispatcher() for _, tc := range testCases { - require.Equal(t, tc.expectPartition, p.DispatchRowChangedEvent(tc.row, 16)) + index, _ := p.DispatchRowChangedEvent(tc.row, 16) + require.Equal(t, tc.expectPartition, index) } } diff --git a/cdc/sink/dmlsink/mq/dispatcher/partition/ts.go b/cdc/sink/dmlsink/mq/dispatcher/partition/ts.go index d974616e142..9ba3dee3a35 100644 --- a/cdc/sink/dmlsink/mq/dispatcher/partition/ts.go +++ b/cdc/sink/dmlsink/mq/dispatcher/partition/ts.go @@ -13,7 +13,11 @@ package partition -import "github.com/pingcap/tiflow/cdc/model" +import ( + "fmt" + + "github.com/pingcap/tiflow/cdc/model" +) // TsDispatcher is a partition dispatcher which dispatch events based on ts. type TsDispatcher struct{} @@ -25,6 +29,6 @@ func NewTsDispatcher() *TsDispatcher { // DispatchRowChangedEvent returns the target partition to which // a row changed event should be dispatched. -func (t *TsDispatcher) DispatchRowChangedEvent(row *model.RowChangedEvent, partitionNum int32) int32 { - return int32(row.CommitTs % uint64(partitionNum)) +func (t *TsDispatcher) DispatchRowChangedEvent(row *model.RowChangedEvent, partitionNum int32) (int32, string) { + return int32(row.CommitTs % uint64(partitionNum)), fmt.Sprintf("%d", row.CommitTs) } diff --git a/cdc/sink/dmlsink/mq/dispatcher/partition/ts_test.go b/cdc/sink/dmlsink/mq/dispatcher/partition/ts_test.go index 1e50eb74e45..c73615c458f 100644 --- a/cdc/sink/dmlsink/mq/dispatcher/partition/ts_test.go +++ b/cdc/sink/dmlsink/mq/dispatcher/partition/ts_test.go @@ -72,6 +72,7 @@ func TestTsDispatcher(t *testing.T) { } p := &TsDispatcher{} for _, tc := range testCases { - require.Equal(t, tc.expectPartition, p.DispatchRowChangedEvent(tc.row, 16)) + index, _ := p.DispatchRowChangedEvent(tc.row, 16) + require.Equal(t, tc.expectPartition, index) } } diff --git a/cdc/sink/dmlsink/mq/dmlproducer/pulsar_dml_producer.go b/cdc/sink/dmlsink/mq/dmlproducer/pulsar_dml_producer.go index 6b11a13b89a..80594ddb5ab 100644 --- a/cdc/sink/dmlsink/mq/dmlproducer/pulsar_dml_producer.go +++ b/cdc/sink/dmlsink/mq/dmlproducer/pulsar_dml_producer.go @@ -72,14 +72,14 @@ func NewPulsarDMLProducer( errCh chan error, failpointCh chan error, ) (DMLProducer, error) { - log.Info("Starting pulsar DML producer ...", + log.Info("Creating pulsar DML producer ...", zap.String("namespace", changefeedID.Namespace), zap.String("changefeed", changefeedID.ID)) + start := time.Now() var pulsarConfig *config.PulsarConfig if sinkConfig.PulsarConfig == nil { - log.Error("new pulsar DML producer fail", - zap.Any("sink:pulsar config is empty", sinkConfig.PulsarConfig)) + log.Error("new pulsar DML producer fail,sink:pulsar config is empty") return nil, cerror.ErrPulsarInvalidConfig. GenWithStackByArgs("pulsar config is empty") } @@ -119,7 +119,8 @@ func NewPulsarDMLProducer( failpointCh: failpointCh, errChan: errCh, } - + log.Info("Pulsar DML producer created", zap.Stringer("changefeed", p.id), + zap.Duration("duration", time.Since(start))) return p, nil } @@ -169,7 +170,16 @@ func (p *pulsarDMLProducer) AsyncSendMessage( zap.String("changefeed", p.id.ID), zap.Error(err)) mq.IncPublishedDMLFail(topic, p.id.ID, message.GetSchema()) - p.errChan <- e + // use this select to avoid send error to a closed channel + // the ctx will always be called before the errChan is closed + select { + case <-ctx.Done(): + return + case p.errChan <- e: + default: + log.Warn("Error channel is full in pulsar DML producer", + zap.Stringer("changefeed", p.id), zap.Error(e)) + } } else if message.Callback != nil { // success message.Callback() diff --git a/cdc/sink/dmlsink/mq/kafka_dml_sink.go b/cdc/sink/dmlsink/mq/kafka_dml_sink.go index 6295226bccb..c4c6bfc0f39 100644 --- a/cdc/sink/dmlsink/mq/kafka_dml_sink.go +++ b/cdc/sink/dmlsink/mq/kafka_dml_sink.go @@ -25,10 +25,10 @@ import ( "github.com/pingcap/tiflow/cdc/sink/util" "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/sink" "github.com/pingcap/tiflow/pkg/sink/codec" "github.com/pingcap/tiflow/pkg/sink/codec/builder" "github.com/pingcap/tiflow/pkg/sink/kafka" - "github.com/pingcap/tiflow/pkg/sink/kafka/claimcheck" tiflowutil "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" ) @@ -91,7 +91,8 @@ func NewKafkaDMLSink( return nil, errors.Trace(err) } - eventRouter, err := dispatcher.NewEventRouter(replicaConfig, topic, sinkURI.Scheme) + scheme := sink.GetScheme(sinkURI) + eventRouter, err := dispatcher.NewEventRouter(replicaConfig, protocol, topic, scheme) if err != nil { return nil, errors.Trace(err) } @@ -103,26 +104,7 @@ func NewKafkaDMLSink( encoderBuilder, err := builder.NewRowEventEncoderBuilder(ctx, encoderConfig) if err != nil { - return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, err) - } - - var ( - claimCheckStorage *claimcheck.ClaimCheck - claimCheckEncoder codec.ClaimCheckLocationEncoder - ok bool - ) - - if encoderConfig.LargeMessageHandle.EnableClaimCheck() { - claimCheckEncoder, ok = encoderBuilder.Build().(codec.ClaimCheckLocationEncoder) - if !ok { - return nil, cerror.ErrKafkaInvalidConfig. - GenWithStack("claim-check enabled but the encoding protocol %s does not support", protocol.String()) - } - - claimCheckStorage, err = claimcheck.New(ctx, encoderConfig.LargeMessageHandle.ClaimCheckStorageURI, changefeedID) - if err != nil { - return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, err) - } + return nil, cerror.WrapError(cerror.ErrKafkaNewProducer, err) } failpointCh := make(chan error, 1) @@ -136,12 +118,10 @@ func NewKafkaDMLSink( concurrency := tiflowutil.GetOrZero(replicaConfig.Sink.EncoderConcurrency) encoderGroup := codec.NewEncoderGroup(encoderBuilder, concurrency, changefeedID) s := newDMLSink(ctx, changefeedID, dmlProducer, adminClient, topicManager, - eventRouter, encoderGroup, protocol, claimCheckStorage, claimCheckEncoder, errCh, - ) + eventRouter, encoderGroup, protocol, scheme, errCh) log.Info("DML sink producer created", zap.String("namespace", changefeedID.Namespace), - zap.String("changefeedID", changefeedID.ID), - zap.Any("options", options)) + zap.String("changefeedID", changefeedID.ID)) return s, nil } diff --git a/cdc/sink/dmlsink/mq/manager/pulsar_manager.go b/cdc/sink/dmlsink/mq/manager/pulsar_manager.go index 9bc686e4a46..25ef163a419 100644 --- a/cdc/sink/dmlsink/mq/manager/pulsar_manager.go +++ b/cdc/sink/dmlsink/mq/manager/pulsar_manager.go @@ -48,11 +48,11 @@ func NewPulsarTopicManager( return mgr, nil } -// GetPartitionNum spend more time,but no use. -// Neither synchronous nor asynchronous sending of pulsar will use PartitionNum -// but this method is used in mq_ddl_sink.go, so an empty implementation is required +// GetPartitionNum always return 1 because we pass a message key to pulsar producer, +// and pulsar producer will hash the key to a partition. +// This method is only used to meet the requirement of mq sink's interface. func (m *pulsarTopicManager) GetPartitionNum(ctx context.Context, topic string) (int32, error) { - return 0, nil + return 1, nil } // CreateTopicAndWaitUntilVisible no need to create first diff --git a/cdc/sink/dmlsink/mq/mq_dml_sink.go b/cdc/sink/dmlsink/mq/mq_dml_sink.go index 6c029b3160a..b75cd0eefed 100644 --- a/cdc/sink/dmlsink/mq/mq_dml_sink.go +++ b/cdc/sink/dmlsink/mq/mq_dml_sink.go @@ -28,9 +28,7 @@ import ( "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/sink" "github.com/pingcap/tiflow/pkg/sink/codec" - "github.com/pingcap/tiflow/pkg/sink/codec/common" "github.com/pingcap/tiflow/pkg/sink/kafka" - "github.com/pingcap/tiflow/pkg/sink/kafka/claimcheck" "go.uber.org/atomic" ) @@ -65,6 +63,8 @@ type dmlSink struct { wg sync.WaitGroup dead chan struct{} + + scheme string } func newDMLSink( @@ -76,14 +76,12 @@ func newDMLSink( eventRouter *dispatcher.EventRouter, encoderGroup codec.EncoderGroup, protocol config.Protocol, - claimCheck *claimcheck.ClaimCheck, - claimCheckEncoder codec.ClaimCheckLocationEncoder, + scheme string, errCh chan error, ) *dmlSink { ctx, cancel := context.WithCancel(ctx) statistics := metrics.NewStatistics(ctx, changefeedID, sink.RowSink) - worker := newWorker(changefeedID, protocol, - producer, encoderGroup, claimCheck, claimCheckEncoder, statistics) + worker := newWorker(changefeedID, protocol, producer, encoderGroup, statistics) s := &dmlSink{ id: changefeedID, @@ -92,6 +90,7 @@ func newDMLSink( ctx: ctx, cancel: cancel, dead: make(chan struct{}), + scheme: scheme, } s.alive.eventRouter = eventRouter s.alive.topicManager = topicManager @@ -151,11 +150,11 @@ func (s *dmlSink) WriteEvents(txns ...*dmlsink.CallbackableEvent[*model.SingleTa if err != nil { return errors.Trace(err) } - partition := s.alive.eventRouter.GetPartitionForRowChange(row, partitionNum) + index, key := s.alive.eventRouter.GetPartitionForRowChange(row, partitionNum) // This never be blocked because this is an unbounded channel. s.alive.worker.msgChan.In() <- mqEvent{ key: TopicPartitionKey{ - Topic: topic, Partition: partition, + Topic: topic, Partition: index, PartitionKey: key, }, rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: row, @@ -184,11 +183,14 @@ func (s *dmlSink) Close() { if s.adminClient != nil { s.adminClient.Close() } - - common.CleanMetrics(s.id) } // Dead checks whether it's dead or not. func (s *dmlSink) Dead() <-chan struct{} { return s.dead } + +// Scheme returns the scheme of this sink. +func (s *dmlSink) Scheme() string { + return s.scheme +} diff --git a/cdc/sink/dmlsink/mq/pulsar_dml_sink.go b/cdc/sink/dmlsink/mq/pulsar_dml_sink.go index 019bede6c0a..93b40f9ac8b 100644 --- a/cdc/sink/dmlsink/mq/pulsar_dml_sink.go +++ b/cdc/sink/dmlsink/mq/pulsar_dml_sink.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tiflow/cdc/sink/util" "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/sink" "github.com/pingcap/tiflow/pkg/sink/codec" "github.com/pingcap/tiflow/pkg/sink/codec/builder" pulsarConfig "github.com/pingcap/tiflow/pkg/sink/pulsar" @@ -58,6 +59,11 @@ func NewPulsarDMLSink( if err != nil { return nil, errors.Trace(err) } + if !util.IsPulsarSupportedProtocols(protocol) { + return nil, cerror.ErrSinkURIInvalid. + GenWithStackByArgs("unsupported protocol, " + + "pulsar sink currently only support these protocols: [canal-json, canal, maxwell]") + } pConfig, err := pulsarConfig.NewPulsarConfig(sinkURI, replicaConfig.Sink.PulsarConfig) if err != nil { @@ -71,10 +77,11 @@ func NewPulsarDMLSink( } failpointCh := make(chan error, 1) - log.Info("Try to create a DML sink producer", zap.Any("pulsar", pConfig)) + log.Info("Try to create a DML sink producer", zap.String("changefeed", changefeedID.String())) start := time.Now() p, err := producerCreator(ctx, changefeedID, client, replicaConfig.Sink, errCh, failpointCh) log.Info("DML sink producer created", + zap.String("changefeed", changefeedID.String()), zap.Duration("duration", time.Since(start))) if err != nil { defer func() { @@ -85,14 +92,14 @@ func NewPulsarDMLSink( return nil, cerror.WrapError(cerror.ErrPulsarNewProducer, err) } + scheme := sink.GetScheme(sinkURI) // The topicManager is not actually used in pulsar , it is only used to create dmlSink. // TODO: Find a way to remove it in newDMLSink. topicManager, err := pulsarTopicManagerCreator(pConfig, client) if err != nil { return nil, errors.Trace(err) } - - eventRouter, err := dispatcher.NewEventRouter(replicaConfig, defaultTopic, sinkURI.Scheme) + eventRouter, err := dispatcher.NewEventRouter(replicaConfig, protocol, defaultTopic, scheme) if err != nil { return nil, errors.Trace(err) } @@ -111,8 +118,8 @@ func NewPulsarDMLSink( concurrency := tiflowutil.GetOrZero(replicaConfig.Sink.EncoderConcurrency) encoderGroup := codec.NewEncoderGroup(encoderBuilder, concurrency, changefeedID) - s := newDMLSink(ctx, changefeedID, p, nil, topicManager, eventRouter, encoderGroup, - protocol, nil, nil, errCh) + s := newDMLSink(ctx, changefeedID, p, nil, topicManager, + eventRouter, encoderGroup, protocol, scheme, errCh) return s, nil } diff --git a/cdc/sink/dmlsink/mq/worker.go b/cdc/sink/dmlsink/mq/worker.go index c9e2e307f24..5e525ba55c5 100644 --- a/cdc/sink/dmlsink/mq/worker.go +++ b/cdc/sink/dmlsink/mq/worker.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tiflow/pkg/chann" "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/sink/codec" - "github.com/pingcap/tiflow/pkg/sink/kafka/claimcheck" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" "golang.org/x/sync/errgroup" @@ -44,8 +43,9 @@ const ( // TopicPartitionKey contains the topic and partition key of the message. type TopicPartitionKey struct { - Topic string - Partition int32 + Topic string + Partition int32 + PartitionKey string } // mqEvent is the event of the mq worker. @@ -67,9 +67,6 @@ type worker struct { // ticker used to force flush the messages when the interval is reached. ticker *time.Ticker - // claimCheckEncoder is used to encode message which has claim-check location, send to kafka. - claimCheckEncoder codec.ClaimCheckLocationEncoder - encoderGroup codec.EncoderGroup // producer is used to send the messages to the Kafka broker. @@ -83,8 +80,6 @@ type worker struct { metricMQWorkerBatchDuration prometheus.Observer // statistics is used to record DML metrics. statistics *metrics.Statistics - - claimCheck *claimcheck.ClaimCheck } // newWorker creates a new flush worker. @@ -93,8 +88,6 @@ func newWorker( protocol config.Protocol, producer dmlproducer.DMLProducer, encoderGroup codec.EncoderGroup, - claimCheck *claimcheck.ClaimCheck, - claimCheckEncoder codec.ClaimCheckLocationEncoder, statistics *metrics.Statistics, ) *worker { w := &worker{ @@ -104,8 +97,6 @@ func newWorker( ticker: time.NewTicker(flushInterval), encoderGroup: encoderGroup, producer: producer, - claimCheck: claimCheck, - claimCheckEncoder: claimCheckEncoder, metricMQWorkerSendMessageDuration: mq.WorkerSendMessageDuration.WithLabelValues(id.Namespace, id.ID), metricMQWorkerBatchSize: mq.WorkerBatchSize.WithLabelValues(id.Namespace, id.ID), metricMQWorkerBatchDuration: mq.WorkerBatchDuration.WithLabelValues(id.Namespace, id.ID), @@ -169,7 +160,12 @@ func (w *worker) nonBatchEncodeRun(ctx context.Context) error { zap.Any("event", event)) continue } - if err := w.encoderGroup.AddEvents(ctx, event.key.Topic, event.key.Partition, event.rowEvent); err != nil { + if err := w.encoderGroup.AddEvents( + ctx, + event.key.Topic, + event.key.Partition, + event.key.PartitionKey, + event.rowEvent); err != nil { return errors.Trace(err) } } @@ -200,7 +196,8 @@ func (w *worker) batchEncodeRun(ctx context.Context) (retErr error) { msgs := eventsBuf[:endIndex] partitionedRows := w.group(msgs) for key, events := range partitionedRows { - if err := w.encoderGroup.AddEvents(ctx, key.Topic, key.Partition, events...); err != nil { + if err := w.encoderGroup. + AddEvents(ctx, key.Topic, key.Partition, key.PartitionKey, events...); err != nil { return errors.Trace(err) } } @@ -212,7 +209,7 @@ func (w *worker) batch( ctx context.Context, events []mqEvent, flushInterval time.Duration, ) (int, error) { index := 0 - max := len(events) + maxBatchSize := len(events) // We need to receive at least one message or be interrupted, // otherwise it will lead to idling. select { @@ -248,7 +245,7 @@ func (w *worker) batch( index++ } - if index >= max { + if index >= maxBatchSize { return index, nil } case <-w.ticker.C: @@ -306,28 +303,14 @@ func (w *worker) sendMessages(ctx context.Context) error { return errors.Trace(err) } for _, message := range future.Messages { - // w.claimCheck in pulsar is nil - if message.ClaimCheckFileName != "" && w.claimCheck != nil { - // send the message to the external storage. - if err = w.claimCheck.WriteMessage(ctx, message); err != nil { - log.Error("send message to the external claim check storage failed", - zap.String("namespace", w.changeFeedID.Namespace), - zap.String("changefeed", w.changeFeedID.ID), - zap.String("filename", message.ClaimCheckFileName), - zap.Error(err)) - return errors.Trace(err) - } - // create the location message which contain the external storage location of the message. - locationMessage, err := w.claimCheckEncoder.NewClaimCheckLocationMessage(message) - if err != nil { - return errors.Trace(err) - } - message = locationMessage - } - // normal message, just send it to the kafka. start := time.Now() if err = w.statistics.RecordBatchExecution(func() (int, error) { - if err := w.producer.AsyncSendMessage(ctx, future.Topic, future.Partition, message); err != nil { + message.SetPartitionKey(future.PartitionKey) + if err := w.producer.AsyncSendMessage( + ctx, + future.Topic, + future.Partition, + message); err != nil { return 0, err } return message.GetRowsCount(), nil @@ -343,10 +326,6 @@ func (w *worker) sendMessages(ctx context.Context) error { func (w *worker) close() { w.msgChan.CloseAndDrain() w.producer.Close() - if w.claimCheck != nil { - w.claimCheck.Close() - } - mq.WorkerSendMessageDuration.DeleteLabelValues(w.changeFeedID.Namespace, w.changeFeedID.ID) mq.WorkerBatchSize.DeleteLabelValues(w.changeFeedID.Namespace, w.changeFeedID.ID) mq.WorkerBatchDuration.DeleteLabelValues(w.changeFeedID.Namespace, w.changeFeedID.ID) diff --git a/cdc/sink/dmlsink/mq/worker_test.go b/cdc/sink/dmlsink/mq/worker_test.go index 6af6d09acc2..1ea4fc420ca 100644 --- a/cdc/sink/dmlsink/mq/worker_test.go +++ b/cdc/sink/dmlsink/mq/worker_test.go @@ -43,7 +43,7 @@ func newBatchEncodeWorker(ctx context.Context, t *testing.T) (*worker, dmlproduc encoderConcurrency := 4 statistics := metrics.NewStatistics(ctx, id, sink.RowSink) encoderGroup := codec.NewEncoderGroup(encoderBuilder, encoderConcurrency, id) - return newWorker(id, config.ProtocolOpen, p, encoderGroup, nil, nil, statistics), p + return newWorker(id, config.ProtocolOpen, p, encoderGroup, statistics), p } func newNonBatchEncodeWorker(ctx context.Context, t *testing.T) (*worker, dmlproducer.DMLProducer) { @@ -57,7 +57,7 @@ func newNonBatchEncodeWorker(ctx context.Context, t *testing.T) (*worker, dmlpro encoderConcurrency := 4 statistics := metrics.NewStatistics(ctx, id, sink.RowSink) encoderGroup := codec.NewEncoderGroup(encoderBuilder, encoderConcurrency, id) - return newWorker(id, config.ProtocolOpen, p, encoderGroup, nil, nil, statistics), p + return newWorker(id, config.ProtocolOpen, p, encoderGroup, statistics), p } func TestNonBatchEncode_SendMessages(t *testing.T) { diff --git a/cdc/sink/dmlsink/txn/mysql/mysql.go b/cdc/sink/dmlsink/txn/mysql/mysql.go index b98f6999d12..f0eb05709a2 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql.go @@ -195,8 +195,7 @@ func NewMySQLBackends( log.Info("MySQL backends is created", zap.String("changefeed", changefeed), zap.Int("workerCount", cfg.WorkerCount), - zap.Bool("forceReplicate", cfg.ForceReplicate), - zap.Bool("enableOldValue", cfg.EnableOldValue)) + zap.Bool("forceReplicate", cfg.ForceReplicate)) return backends, nil } @@ -530,9 +529,8 @@ func (s *mysqlBackend) prepareDMLs() *preparedDMLs { values := make([][]interface{}, 0, s.rows) callbacks := make([]dmlsink.CallbackFunc, 0, len(s.events)) - // translateToInsert control the update and insert behavior - // we only translate into insert when old value is enabled and safe mode is disabled - translateToInsert := s.cfg.EnableOldValue && !s.cfg.SafeMode + // translateToInsert control the update and insert behavior. + translateToInsert := !s.cfg.SafeMode rowCount := 0 approximateSize := int64(0) @@ -555,7 +553,6 @@ func (s *mysqlBackend) prepareDMLs() *preparedDMLs { zap.Bool("translateToInsert", translateToInsert), zap.Uint64("firstRowCommitTs", firstRow.CommitTs), zap.Uint64("firstRowReplicatingTs", firstRow.ReplicatingTs), - zap.Bool("enableOldValue", s.cfg.EnableOldValue), zap.Bool("safeMode", s.cfg.SafeMode)) if event.Callback != nil { diff --git a/cdc/sink/dmlsink/txn/mysql/mysql_test.go b/cdc/sink/dmlsink/txn/mysql/mysql_test.go index ef321f07358..461641dbc38 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql_test.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql_test.go @@ -112,23 +112,29 @@ func TestPrepareDML(t *testing.T) { sqls: []string{}, values: [][]interface{}{}, }, - }, { + }, + // delete event + { input: []*model.RowChangedEvent{ { StartTs: 418658114257813514, CommitTs: 418658114257813515, Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, - PreColumns: []*model.Column{nil, { - Name: "a1", - Type: mysql.TypeLong, - Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, - Value: 1, - }, { - Name: "a3", - Type: mysql.TypeLong, - Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, - Value: 1, - }}, + PreColumns: []*model.Column{ + nil, + { + Name: "a1", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, + Value: 1, + }, + { + Name: "a3", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, + Value: 1, + }, + }, IndexColumns: [][]int{{1, 2}}, }, }, @@ -139,35 +145,42 @@ func TestPrepareDML(t *testing.T) { rowCount: 1, approximateSize: 74, }, - }, { + }, + // insert event. + { input: []*model.RowChangedEvent{ { StartTs: 418658114257813516, CommitTs: 418658114257813517, Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, - Columns: []*model.Column{nil, { - Name: "a1", - Type: mysql.TypeLong, - Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, - Value: 2, - }, { - Name: "a3", - Type: mysql.TypeLong, - Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, - Value: 2, - }}, + Columns: []*model.Column{ + nil, + { + Name: "a1", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, + Value: 2, + }, + { + Name: "a3", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, + Value: 2, + }, + }, IndexColumns: [][]int{{1, 2}}, }, }, expected: &preparedDMLs{ startTs: []model.Ts{418658114257813516}, - sqls: []string{"REPLACE INTO `common_1`.`uk_without_pk` (`a1`,`a3`) VALUES (?,?)"}, + sqls: []string{"INSERT INTO `common_1`.`uk_without_pk` (`a1`,`a3`) VALUES (?,?)"}, values: [][]interface{}{{2, 2}}, rowCount: 1, - approximateSize: 64, + approximateSize: 63, }, }, } + ctx, cancel := context.WithCancel(context.Background()) defer cancel() ms := newMySQLBackendWithoutDB(ctx) @@ -1153,7 +1166,6 @@ func TestMysqlSinkSafeModeOff(t *testing.T) { defer cancel() ms := newMySQLBackendWithoutDB(ctx) ms.cfg.SafeMode = false - ms.cfg.EnableOldValue = true for _, tc := range testCases { ms.events = make([]*dmlsink.TxnCallbackableEvent, 1) ms.events[0] = &dmlsink.TxnCallbackableEvent{ @@ -1625,7 +1637,6 @@ func TestPrepareBatchDMLs(t *testing.T) { ms := newMySQLBackendWithoutDB(ctx) ms.cfg.BatchDMLEnable = true ms.cfg.SafeMode = false - ms.cfg.EnableOldValue = true for _, tc := range testCases { ms.cfg.IsTiDB = tc.isTiDB ms.events = make([]*dmlsink.TxnCallbackableEvent, 1) diff --git a/cdc/sink/dmlsink/txn/txn_dml_sink.go b/cdc/sink/dmlsink/txn/txn_dml_sink.go index a96aa9f60f7..b8f78059f5d 100644 --- a/cdc/sink/dmlsink/txn/txn_dml_sink.go +++ b/cdc/sink/dmlsink/txn/txn_dml_sink.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tiflow/cdc/sink/tablesink/state" "github.com/pingcap/tiflow/pkg/causality" "github.com/pingcap/tiflow/pkg/config" - psink "github.com/pingcap/tiflow/pkg/sink" + "github.com/pingcap/tiflow/pkg/sink" pmysql "github.com/pingcap/tiflow/pkg/sink/mysql" "golang.org/x/sync/errgroup" ) @@ -54,6 +54,8 @@ type dmlSink struct { dead chan struct{} statistics *metrics.Statistics + + scheme string } // GetDBConnImpl is the implementation of pmysql.Factory. @@ -72,7 +74,7 @@ func NewMySQLSink( conflictDetectorSlots uint64, ) (*dmlSink, error) { ctx, cancel := context.WithCancel(ctx) - statistics := metrics.NewStatistics(ctx, changefeedID, psink.TxnSink) + statistics := metrics.NewStatistics(ctx, changefeedID, sink.TxnSink) backendImpls, err := mysql.NewMySQLBackends(ctx, changefeedID, sinkURI, replicaConfig, GetDBConnImpl, statistics) if err != nil { @@ -84,11 +86,13 @@ func NewMySQLSink( for _, impl := range backendImpls { backends = append(backends, impl) } - sink := newSink(ctx, changefeedID, backends, errCh, conflictDetectorSlots) - sink.statistics = statistics - sink.cancel = cancel - return sink, nil + s := newSink(ctx, changefeedID, backends, errCh, conflictDetectorSlots) + s.statistics = statistics + s.cancel = cancel + s.scheme = sink.GetScheme(sinkURI) + + return s, nil } func newSink(ctx context.Context, @@ -173,3 +177,7 @@ func (s *dmlSink) Close() { func (s *dmlSink) Dead() <-chan struct{} { return s.dead } + +func (s *dmlSink) Scheme() string { + return s.scheme +} diff --git a/cdc/sink/tablesink/table_sink_impl.go b/cdc/sink/tablesink/table_sink_impl.go index 332cccba64c..da067501727 100644 --- a/cdc/sink/tablesink/table_sink_impl.go +++ b/cdc/sink/tablesink/table_sink_impl.go @@ -110,7 +110,7 @@ func (e *EventTableSink[E, P]) UpdateResolvedTs(resolvedTs model.ResolvedTs) err resolvedCallbackableEvents := make([]*dmlsink.CallbackableEvent[E], 0, len(resolvedEvents)) for _, ev := range resolvedEvents { - if err := ev.TrySplitAndSortUpdateEvent(); err != nil { + if err := ev.TrySplitAndSortUpdateEvent(e.backendSink.Scheme()); err != nil { return SinkInternalError{err} } // We have to record the event ID for the callback. diff --git a/cdc/sink/tablesink/table_sink_impl_test.go b/cdc/sink/tablesink/table_sink_impl_test.go index 86f2e79e061..95a26711a25 100644 --- a/cdc/sink/tablesink/table_sink_impl_test.go +++ b/cdc/sink/tablesink/table_sink_impl_test.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink/dmlsink" "github.com/pingcap/tiflow/cdc/sink/tablesink/state" + "github.com/pingcap/tiflow/pkg/sink" "github.com/pingcap/tiflow/pkg/spanz" "github.com/prometheus/client_golang/prometheus" "github.com/stretchr/testify/require" @@ -48,6 +49,10 @@ func (m *mockEventSink) Dead() <-chan struct{} { return m.dead } +func (m *mockEventSink) Scheme() string { + return sink.BlackHoleScheme +} + // acknowledge the txn events by call the callback function. func (m *mockEventSink) acknowledge(commitTs uint64) []*dmlsink.TxnCallbackableEvent { var droppedEvents []*dmlsink.TxnCallbackableEvent diff --git a/cdc/sink/util/helper.go b/cdc/sink/util/helper.go index 23a0660967e..4ff6db3e6d0 100644 --- a/cdc/sink/util/helper.go +++ b/cdc/sink/util/helper.go @@ -106,3 +106,8 @@ func GetTopicManagerAndTryCreateTopic( return topicManager, nil } + +// IsPulsarSupportedProtocols returns whether the protocol is supported by pulsar. +func IsPulsarSupportedProtocols(p config.Protocol) bool { + return p == config.ProtocolCanalJSON +} diff --git a/cdc/syncpointstore/mysql_syncpoint_store.go b/cdc/syncpointstore/mysql_syncpoint_store.go index 48b163558b7..ce4e88df234 100644 --- a/cdc/syncpointstore/mysql_syncpoint_store.go +++ b/cdc/syncpointstore/mysql_syncpoint_store.go @@ -77,7 +77,7 @@ func newMySQLSyncPointStore( return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") } - log.Info("Start mysql syncpoint sink") + log.Info("Start mysql syncpoint sink", zap.String("changefeed", id.String())) return &mysqlSyncPointStore{ db: syncDB, @@ -146,7 +146,7 @@ func (s *mysqlSyncPointStore) SinkSyncPoint(ctx context.Context, var secondaryTs string err = row.Scan(&secondaryTs) if err != nil { - log.Info("sync table: get tidb_current_ts err") + log.Info("sync table: get tidb_current_ts err", zap.String("changefeed", id.String())) err2 := tx.Rollback() if err2 != nil { log.Error("failed to write syncpoint table", zap.Error(err)) diff --git a/cmd/kafka-consumer/main.go b/cmd/kafka-consumer/main.go index f12dadccaf2..dd3718c9ce4 100644 --- a/cmd/kafka-consumer/main.go +++ b/cmd/kafka-consumer/main.go @@ -486,7 +486,7 @@ func NewConsumer(ctx context.Context, o *consumerOption) (*Consumer, error) { } if o.replicaConfig != nil { - eventRouter, err := dispatcher.NewEventRouter(o.replicaConfig, o.topic, "kafka") + eventRouter, err := dispatcher.NewEventRouter(o.replicaConfig, o.protocol, o.topic, "kafka") if err != nil { return nil, errors.Trace(err) } @@ -662,7 +662,7 @@ func (c *Consumer) ConsumeClaim(session sarama.ConsumerGroupSession, claim saram } if c.eventRouter != nil { - target := c.eventRouter.GetPartitionForRowChange(row, c.option.partitionNum) + target, _ := c.eventRouter.GetPartitionForRowChange(row, c.option.partitionNum) if partition != target { log.Panic("RowChangedEvent dispatched to wrong partition", zap.Int32("obtained", partition), @@ -946,7 +946,7 @@ func (g *fakeTableIDGenerator) generateFakeTableID(schema, table string, partiti func openDB(ctx context.Context, dsn string) (*sql.DB, error) { db, err := sql.Open("mysql", dsn) if err != nil { - log.Error("open db failed", zap.String("dsn", dsn), zap.Error(err)) + log.Error("open db failed", zap.Error(err)) return nil, errors.Trace(err) } @@ -957,7 +957,7 @@ func openDB(ctx context.Context, dsn string) (*sql.DB, error) { ctx, cancel := context.WithTimeout(ctx, 5*time.Second) defer cancel() if err = db.PingContext(ctx); err != nil { - log.Error("ping db failed", zap.String("dsn", dsn), zap.Error(err)) + log.Error("ping db failed", zap.Error(err)) return nil, errors.Trace(err) } log.Info("open db success", zap.String("dsn", dsn)) diff --git a/cmd/pulsar-consumer/main.go b/cmd/pulsar-consumer/main.go new file mode 100644 index 00000000000..16f1e0571f3 --- /dev/null +++ b/cmd/pulsar-consumer/main.go @@ -0,0 +1,708 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package main + +import ( + "context" + "fmt" + "math" + "net/url" + "os" + "os/signal" + "sort" + "strconv" + "strings" + "sync" + "sync/atomic" + "syscall" + "time" + + "github.com/apache/pulsar-client-go/pulsar" + "github.com/pingcap/errors" + "github.com/pingcap/log" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sink/ddlsink" + ddlsinkfactory "github.com/pingcap/tiflow/cdc/sink/ddlsink/factory" + eventsinkfactory "github.com/pingcap/tiflow/cdc/sink/dmlsink/factory" + "github.com/pingcap/tiflow/cdc/sink/tablesink" + sutil "github.com/pingcap/tiflow/cdc/sink/util" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/logutil" + "github.com/pingcap/tiflow/pkg/quotes" + "github.com/pingcap/tiflow/pkg/sink" + "github.com/pingcap/tiflow/pkg/sink/codec" + "github.com/pingcap/tiflow/pkg/sink/codec/canal" + "github.com/pingcap/tiflow/pkg/sink/codec/common" + tpulsar "github.com/pingcap/tiflow/pkg/sink/pulsar" + "github.com/pingcap/tiflow/pkg/spanz" + "github.com/pingcap/tiflow/pkg/util" + "github.com/pingcap/tiflow/pkg/version" + "github.com/prometheus/client_golang/prometheus" + "github.com/spf13/cobra" + "go.uber.org/zap" +) + +// ConsumerOption represents the options of the pulsar consumer +type ConsumerOption struct { + address []string + topic string + + protocol config.Protocol + enableTiDBExtension bool + + logPath string + logLevel string + timezone string + ca, cert, key string + + downstreamURI string + partitionNum int +} + +func newConsumerOption() *ConsumerOption { + return &ConsumerOption{ + protocol: config.ProtocolDefault, + } +} + +// Adjust the consumer option by the upstream uri passed in parameters. +func (o *ConsumerOption) Adjust(upstreamURI *url.URL, configFile string) { + // the default value of partitionNum is 1 + o.partitionNum = 1 + + o.topic = strings.TrimFunc(upstreamURI.Path, func(r rune) bool { + return r == '/' + }) + + o.address = strings.Split(upstreamURI.Host, ",") + + s := upstreamURI.Query().Get("protocol") + if s != "" { + protocol, err := config.ParseSinkProtocolFromString(s) + if err != nil { + log.Panic("invalid protocol", zap.Error(err), zap.String("protocol", s)) + } + if !sutil.IsPulsarSupportedProtocols(protocol) { + log.Panic("unsupported protocol, pulsar sink currently only support these protocols: [canal-json, canal, maxwell]", + zap.String("protocol", s)) + } + o.protocol = protocol + } + + s = upstreamURI.Query().Get("enable-tidb-extension") + if s != "" { + enableTiDBExtension, err := strconv.ParseBool(s) + if err != nil { + log.Panic("invalid enable-tidb-extension of upstream-uri") + } + if enableTiDBExtension { + if o.protocol != config.ProtocolCanalJSON && o.protocol != config.ProtocolAvro { + log.Panic("enable-tidb-extension only work with canal-json / avro") + } + } + o.enableTiDBExtension = enableTiDBExtension + } + + log.Info("consumer option adjusted", + zap.String("configFile", configFile), + zap.String("address", strings.Join(o.address, ",")), + zap.String("topic", o.topic), + zap.Any("protocol", o.protocol), + zap.Bool("enableTiDBExtension", o.enableTiDBExtension)) +} + +var ( + upstreamURIStr string + configFile string + consumerOption = newConsumerOption() +) + +func main() { + cmd := &cobra.Command{ + Use: "pulsar consumer", + Run: run, + } + // Flags for the root command + cmd.Flags().StringVar(&configFile, "config", "", "config file for changefeed") + cmd.Flags().StringVar(&upstreamURIStr, "upstream-uri", "", "pulsar uri") + cmd.Flags().StringVar(&consumerOption.downstreamURI, "downstream-uri", "", "downstream sink uri") + cmd.Flags().StringVar(&consumerOption.timezone, "tz", "System", "Specify time zone of pulsar consumer") + cmd.Flags().StringVar(&consumerOption.ca, "ca", "", "CA certificate path for pulsar SSL connection") + cmd.Flags().StringVar(&consumerOption.cert, "cert", "", "Certificate path for pulsar SSL connection") + cmd.Flags().StringVar(&consumerOption.key, "key", "", "Private key path for pulsar SSL connection") + cmd.Flags().StringVar(&consumerOption.logPath, "log-file", "cdc_pulsar_consumer.log", "log file path") + cmd.Flags().StringVar(&consumerOption.logLevel, "log-level", "info", "log file path") + + if err := cmd.Execute(); err != nil { + fmt.Println(err) + } +} + +func run(cmd *cobra.Command, args []string) { + err := logutil.InitLogger(&logutil.Config{ + Level: consumerOption.logLevel, + File: consumerOption.logPath, + }, + logutil.WithInitGRPCLogger(), + logutil.WithInitSaramaLogger(), + ) + if err != nil { + log.Error("init logger failed", zap.Error(err)) + return + } + + version.LogVersionInfo("pulsar consumer") + + upstreamURI, err := url.Parse(upstreamURIStr) + if err != nil { + log.Panic("invalid upstream-uri", zap.Error(err)) + } + scheme := strings.ToLower(upstreamURI.Scheme) + if !sink.IsPulsarScheme(scheme) { + log.Panic("invalid upstream-uri scheme, the scheme of upstream-uri must be pulsar schema", + zap.String("schema", scheme), + zap.String("upstreamURI", upstreamURIStr)) + } + + consumerOption.Adjust(upstreamURI, configFile) + + ctx, cancel := context.WithCancel(context.Background()) + consumer, err := NewConsumer(ctx, consumerOption) + if err != nil { + log.Panic("Error creating pulsar consumer", zap.Error(err)) + } + + pulsarConsumer, client := NewPulsarConsumer(consumerOption) + defer client.Close() + defer pulsarConsumer.Close() + msgChan := pulsarConsumer.Chan() + + wg := &sync.WaitGroup{} + wg.Add(1) + go func() { + defer wg.Done() + for { + select { + case <-ctx.Done(): + log.Info("terminating: context cancelled") + return + case consumerMsg := <-msgChan: + log.Debug(fmt.Sprintf("Received message msgId: %#v -- content: '%s'\n", + consumerMsg.ID(), + string(consumerMsg.Payload()))) + err := consumer.HandleMsg(consumerMsg.Message) + if err != nil { + log.Panic("Error consuming message", zap.Error(err)) + } + err = pulsarConsumer.AckID(consumerMsg.Message.ID()) + if err != nil { + log.Panic("Error ack message", zap.Error(err)) + } + } + } + }() + + wg.Add(1) + go func() { + defer wg.Done() + if err := consumer.Run(ctx); err != nil { + if err != context.Canceled { + log.Panic("Error running consumer", zap.Error(err)) + } + } + }() + + log.Info("TiCDC consumer up and running!...") + sigterm := make(chan os.Signal, 1) + signal.Notify(sigterm, syscall.SIGINT, syscall.SIGTERM) + select { + case <-ctx.Done(): + log.Info("terminating: context cancelled") + case <-sigterm: + log.Info("terminating: via signal") + } + cancel() + wg.Wait() +} + +// NewPulsarConsumer creates a pulsar consumer +func NewPulsarConsumer(option *ConsumerOption) (pulsar.Consumer, pulsar.Client) { + pulsarURL := "pulsar" + "://" + option.address[0] + topicName := option.topic + subscriptionName := "pulsar-test-subscription" + + client, err := pulsar.NewClient(pulsar.ClientOptions{ + URL: pulsarURL, + Logger: tpulsar.NewPulsarLogger(), + }) + if err != nil { + log.Fatal("can't create pulsar client: %v", zap.Error(err)) + } + + consumerConfig := pulsar.ConsumerOptions{ + Topic: topicName, + SubscriptionName: subscriptionName, + Type: pulsar.Exclusive, + } + + consumer, err := client.Subscribe(consumerConfig) + if err != nil { + log.Fatal("can't create pulsar consumer: %v", zap.Error(err)) + } + return consumer, client +} + +// partitionSinks maintained for each partition, it may sync data for multiple tables. +type partitionSinks struct { + tablesCommitTsMap sync.Map + tableSinksMap sync.Map + // resolvedTs record the maximum timestamp of the received event + resolvedTs uint64 +} + +// Consumer represents a local pulsar consumer +type Consumer struct { + eventGroups map[int64]*eventsGroup + ddlList []*model.DDLEvent + ddlListMu sync.Mutex + lastReceivedDDL *model.DDLEvent + ddlSink ddlsink.Sink + fakeTableIDGenerator *fakeTableIDGenerator + + // sinkFactory is used to create table sink for each table. + sinkFactory *eventsinkfactory.SinkFactory + sinks []*partitionSinks + sinksMu sync.Mutex + + // initialize to 0 by default + globalResolvedTs uint64 + + tz *time.Location + + codecConfig *common.Config + + option *ConsumerOption +} + +// NewConsumer creates a new cdc pulsar consumer +// the consumer is responsible for consuming the data from the pulsar topic +// and write the data to the downstream. +func NewConsumer(ctx context.Context, o *ConsumerOption) (*Consumer, error) { + c := new(Consumer) + c.option = o + + tz, err := util.GetTimezone(o.timezone) + if err != nil { + return nil, errors.Annotate(err, "can not load timezone") + } + config.GetGlobalServerConfig().TZ = o.timezone + c.tz = tz + + c.fakeTableIDGenerator = &fakeTableIDGenerator{ + tableIDs: make(map[string]int64), + } + + c.codecConfig = common.NewConfig(o.protocol) + c.codecConfig.EnableTiDBExtension = o.enableTiDBExtension + if c.codecConfig.Protocol == config.ProtocolAvro { + c.codecConfig.AvroEnableWatermark = true + } + + c.sinks = make([]*partitionSinks, o.partitionNum) + ctx, cancel := context.WithCancel(ctx) + errChan := make(chan error, 1) + for i := 0; i < o.partitionNum; i++ { + c.sinks[i] = &partitionSinks{} + } + + changefeedID := model.DefaultChangeFeedID("pulsar-consumer") + f, err := eventsinkfactory.New(ctx, changefeedID, o.downstreamURI, config.GetDefaultReplicaConfig(), errChan) + if err != nil { + cancel() + return nil, errors.Trace(err) + } + c.sinkFactory = f + + go func() { + err := <-errChan + if errors.Cause(err) != context.Canceled { + log.Error("error on running consumer", zap.Error(err)) + } else { + log.Info("consumer exited") + } + cancel() + }() + + ddlSink, err := ddlsinkfactory.New(ctx, changefeedID, o.downstreamURI, config.GetDefaultReplicaConfig()) + if err != nil { + cancel() + return nil, errors.Trace(err) + } + c.ddlSink = ddlSink + c.eventGroups = make(map[int64]*eventsGroup) + return c, nil +} + +type eventsGroup struct { + events []*model.RowChangedEvent +} + +func newEventsGroup() *eventsGroup { + return &eventsGroup{ + events: make([]*model.RowChangedEvent, 0), + } +} + +func (g *eventsGroup) Append(e *model.RowChangedEvent) { + g.events = append(g.events, e) +} + +func (g *eventsGroup) Resolve(resolveTs uint64) []*model.RowChangedEvent { + sort.Slice(g.events, func(i, j int) bool { + return g.events[i].CommitTs < g.events[j].CommitTs + }) + + i := sort.Search(len(g.events), func(i int) bool { + return g.events[i].CommitTs > resolveTs + }) + result := g.events[:i] + g.events = g.events[i:] + + return result +} + +// HandleMsg handles the message received from the pulsar consumer +func (c *Consumer) HandleMsg(msg pulsar.Message) error { + c.sinksMu.Lock() + sink := c.sinks[0] + c.sinksMu.Unlock() + if sink == nil { + panic("sink should initialized") + } + + ctx := context.Background() + var ( + decoder codec.RowEventDecoder + err error + ) + + switch c.codecConfig.Protocol { + case config.ProtocolCanalJSON: + decoder, err = canal.NewBatchDecoder(ctx, c.codecConfig, nil) + if err != nil { + return err + } + default: + log.Panic("Protocol not supported", zap.Any("Protocol", c.codecConfig.Protocol)) + } + if err != nil { + return errors.Trace(err) + } + + if err := decoder.AddKeyValue([]byte(msg.Key()), msg.Payload()); err != nil { + log.Error("add key value to the decoder failed", zap.Error(err)) + return errors.Trace(err) + } + + counter := 0 + for { + tp, hasNext, err := decoder.HasNext() + if err != nil { + log.Panic("decode message key failed", zap.Error(err)) + } + if !hasNext { + break + } + + counter++ + switch tp { + case model.MessageTypeDDL: + // for some protocol, DDL would be dispatched to all partitions, + // Consider that DDL a, b, c received from partition-0, the latest DDL is c, + // if we receive `a` from partition-1, which would be seemed as DDL regression, + // then cause the consumer panic, but it was a duplicate one. + // so we only handle DDL received from partition-0 should be enough. + // but all DDL event messages should be consumed. + ddl, err := decoder.NextDDLEvent() + if err != nil { + log.Panic("decode message value failed", + zap.ByteString("value", msg.Payload()), + zap.Error(err)) + } + c.appendDDL(ddl) + case model.MessageTypeRow: + row, err := decoder.NextRowChangedEvent() + if err != nil { + log.Panic("decode message value failed", + zap.ByteString("value", msg.Payload()), + zap.Error(err)) + } + globalResolvedTs := atomic.LoadUint64(&c.globalResolvedTs) + partitionResolvedTs := atomic.LoadUint64(&sink.resolvedTs) + if row.CommitTs <= globalResolvedTs || row.CommitTs <= partitionResolvedTs { + log.Warn("RowChangedEvent fallback row, ignore it", + zap.Uint64("commitTs", row.CommitTs), + zap.Uint64("globalResolvedTs", globalResolvedTs), + zap.Uint64("partitionResolvedTs", partitionResolvedTs), + zap.Int32("partition", msg.ID().PartitionIdx()), + zap.Any("row", row)) + // todo: mark the offset after the DDL is fully synced to the downstream mysql. + continue + } + var partitionID int64 + if row.Table.IsPartition { + partitionID = row.Table.TableID + } + // use schema, table and tableID to identify a table + tableID := c.fakeTableIDGenerator. + generateFakeTableID(row.Table.Schema, row.Table.Table, partitionID) + row.Table.TableID = tableID + + group, ok := c.eventGroups[tableID] + if !ok { + group = newEventsGroup() + c.eventGroups[tableID] = group + } + group.Append(row) + case model.MessageTypeResolved: + ts, err := decoder.NextResolvedEvent() + if err != nil { + log.Panic("decode message value failed", + zap.ByteString("value", msg.Payload()), + zap.Error(err)) + } + + globalResolvedTs := atomic.LoadUint64(&c.globalResolvedTs) + partitionResolvedTs := atomic.LoadUint64(&sink.resolvedTs) + if ts < globalResolvedTs || ts < partitionResolvedTs { + log.Warn("partition resolved ts fallback, skip it", + zap.Uint64("ts", ts), + zap.Uint64("partitionResolvedTs", partitionResolvedTs), + zap.Uint64("globalResolvedTs", globalResolvedTs), + zap.Int32("partition", msg.ID().PartitionIdx())) + continue + } + + for tableID, group := range c.eventGroups { + events := group.Resolve(ts) + if len(events) == 0 { + continue + } + if _, ok := sink.tableSinksMap.Load(tableID); !ok { + log.Info("create table sink for consumer", zap.Any("tableID", tableID)) + tableSink := c.sinkFactory.CreateTableSinkForConsumer( + model.DefaultChangeFeedID("pulsar-consumer"), + spanz.TableIDToComparableSpan(tableID), + events[0].CommitTs, + prometheus.NewCounter(prometheus.CounterOpts{})) + + log.Info("table sink created", zap.Any("tableID", tableID), + zap.Any("tableSink", tableSink.GetCheckpointTs())) + + sink.tableSinksMap.Store(tableID, tableSink) + } + s, _ := sink.tableSinksMap.Load(tableID) + s.(tablesink.TableSink).AppendRowChangedEvents(events...) + commitTs := events[len(events)-1].CommitTs + lastCommitTs, ok := sink.tablesCommitTsMap.Load(tableID) + if !ok || lastCommitTs.(uint64) < commitTs { + sink.tablesCommitTsMap.Store(tableID, commitTs) + } + } + atomic.StoreUint64(&sink.resolvedTs, ts) + } + + } + return nil +} + +// append DDL wait to be handled, only consider the constraint among DDLs. +// for DDL a / b received in the order, a.CommitTs < b.CommitTs should be true. +func (c *Consumer) appendDDL(ddl *model.DDLEvent) { + c.ddlListMu.Lock() + defer c.ddlListMu.Unlock() + // DDL CommitTs fallback, just crash it to indicate the bug. + if c.lastReceivedDDL != nil && ddl.CommitTs < c.lastReceivedDDL.CommitTs { + log.Panic("DDL CommitTs < lastReceivedDDL.CommitTs", + zap.Uint64("commitTs", ddl.CommitTs), + zap.Uint64("lastReceivedDDLCommitTs", c.lastReceivedDDL.CommitTs), + zap.Any("DDL", ddl)) + } + + // A rename tables DDL job contains multiple DDL events with same CommitTs. + // So to tell if a DDL is redundant or not, we must check the equivalence of + // the current DDL and the DDL with max CommitTs. + if ddl == c.lastReceivedDDL { + log.Info("ignore redundant DDL, the DDL is equal to ddlWithMaxCommitTs", + zap.Any("DDL", ddl)) + return + } + + c.ddlList = append(c.ddlList, ddl) + log.Info("DDL event received", zap.Any("DDL", ddl)) + c.lastReceivedDDL = ddl +} + +func (c *Consumer) getFrontDDL() *model.DDLEvent { + c.ddlListMu.Lock() + defer c.ddlListMu.Unlock() + if len(c.ddlList) > 0 { + return c.ddlList[0] + } + return nil +} + +func (c *Consumer) popDDL() *model.DDLEvent { + c.ddlListMu.Lock() + defer c.ddlListMu.Unlock() + if len(c.ddlList) > 0 { + ddl := c.ddlList[0] + c.ddlList = c.ddlList[1:] + return ddl + } + return nil +} + +func (c *Consumer) forEachSink(fn func(sink *partitionSinks) error) error { + c.sinksMu.Lock() + defer c.sinksMu.Unlock() + for _, sink := range c.sinks { + if err := fn(sink); err != nil { + return errors.Trace(err) + } + } + return nil +} + +// getMinResolvedTs returns the minimum resolvedTs of all the partitionSinks +func (c *Consumer) getMinResolvedTs() (result uint64, err error) { + result = uint64(math.MaxUint64) + err = c.forEachSink(func(sink *partitionSinks) error { + a := atomic.LoadUint64(&sink.resolvedTs) + if a < result { + result = a + } + return nil + }) + return result, err +} + +// Run the Consumer +func (c *Consumer) Run(ctx context.Context) error { + ticker := time.NewTicker(200 * time.Millisecond) + defer ticker.Stop() + for { + select { + case <-ctx.Done(): + return ctx.Err() + case <-ticker.C: + // 1. Get the minimum resolvedTs of all the partitionSinks + minResolvedTs, err := c.getMinResolvedTs() + if err != nil { + return errors.Trace(err) + } + + // 2. check if there is a DDL event that can be executed + // if there is, execute it and update the minResolvedTs + nextDDL := c.getFrontDDL() + if nextDDL != nil && minResolvedTs >= nextDDL.CommitTs { + // flush DMLs that commitTs <= todoDDL.CommitTs + if err := c.forEachSink(func(sink *partitionSinks) error { + return flushRowChangedEvents(ctx, sink, nextDDL.CommitTs) + }); err != nil { + return errors.Trace(err) + } + + // all DMLs with commitTs <= todoDDL.CommitTs have been flushed to downstream, + // so we can execute the DDL now. + if err := c.ddlSink.WriteDDLEvent(ctx, nextDDL); err != nil { + return errors.Trace(err) + } + ddl := c.popDDL() + log.Info("DDL executed", zap.Any("DDL", ddl)) + } + + // 3. Update global resolved ts + if c.globalResolvedTs > minResolvedTs { + log.Panic("global ResolvedTs fallback", + zap.Uint64("globalResolvedTs", c.globalResolvedTs), + zap.Uint64("minPartitionResolvedTs", minResolvedTs)) + } + + if c.globalResolvedTs < minResolvedTs { + c.globalResolvedTs = minResolvedTs + } + + // 4. flush all the DMLs that commitTs <= globalResolvedTs + if err := c.forEachSink(func(sink *partitionSinks) error { + return flushRowChangedEvents(ctx, sink, c.globalResolvedTs) + }); err != nil { + return errors.Trace(err) + } + } + } +} + +// flushRowChangedEvents flushes all the DMLs that commitTs <= resolvedTs +// Note: This function is synchronous, it will block until all the DMLs are flushed. +func flushRowChangedEvents(ctx context.Context, sink *partitionSinks, resolvedTs uint64) error { + for { + select { + case <-ctx.Done(): + return ctx.Err() + default: + } + flushedResolvedTs := true + sink.tablesCommitTsMap.Range(func(key, value interface{}) bool { + tableID := key.(int64) + resolvedTs := model.NewResolvedTs(resolvedTs) + tableSink, ok := sink.tableSinksMap.Load(tableID) + if !ok { + log.Panic("Table sink not found", zap.Int64("tableID", tableID)) + } + if err := tableSink.(tablesink.TableSink).UpdateResolvedTs(resolvedTs); err != nil { + log.Error("Failed to update resolved ts", zap.Error(err)) + return false + } + if !tableSink.(tablesink.TableSink).GetCheckpointTs().EqualOrGreater(resolvedTs) { + flushedResolvedTs = false + } + return true + }) + if flushedResolvedTs { + return nil + } + } +} + +type fakeTableIDGenerator struct { + tableIDs map[string]int64 + currentTableID int64 + mu sync.Mutex +} + +func (g *fakeTableIDGenerator) generateFakeTableID(schema, table string, partition int64) int64 { + g.mu.Lock() + defer g.mu.Unlock() + key := quotes.QuoteSchema(schema, table) + if partition != 0 { + key = fmt.Sprintf("%s.`%d`", key, partition) + } + if tableID, ok := g.tableIDs[key]; ok { + return tableID + } + g.currentTableID++ + g.tableIDs[key] = g.currentTableID + return g.currentTableID +} diff --git a/deployments/ticdc/docker-compose/configs/canal-test-config.toml b/deployments/ticdc/docker-compose/configs/canal-test-config.toml index 0fc9a85c382..5d8fa34c904 100644 --- a/deployments/ticdc/docker-compose/configs/canal-test-config.toml +++ b/deployments/ticdc/docker-compose/configs/canal-test-config.toml @@ -1,5 +1,3 @@ -enable-old-value = true - [sink] protocol = "canal" diff --git a/deployments/ticdc/docker-compose/configs/enable-oldvalue-config.toml b/deployments/ticdc/docker-compose/configs/enable-oldvalue-config.toml index 4293a79e451..e69de29bb2d 100644 --- a/deployments/ticdc/docker-compose/configs/enable-oldvalue-config.toml +++ b/deployments/ticdc/docker-compose/configs/enable-oldvalue-config.toml @@ -1 +0,0 @@ -enable-old-value = true diff --git a/dm/syncer/opt_sharding_group.go b/dm/syncer/opt_sharding_group.go index d87514371ea..ce4060de642 100644 --- a/dm/syncer/opt_sharding_group.go +++ b/dm/syncer/opt_sharding_group.go @@ -233,3 +233,11 @@ func (k *OptShardingGroupKeeper) RemoveSchema(schema string) { } } } + +// Reset resets the keeper. +func (k *OptShardingGroupKeeper) Reset() { + k.Lock() + defer k.Unlock() + k.groups = make(map[string]*OptShardingGroup) + k.shardingReSyncs = make(map[string]binlog.Location) +} diff --git a/dm/syncer/opt_sharding_group_test.go b/dm/syncer/opt_sharding_group_test.go index 88ad02cb873..9cd4eec7e6e 100644 --- a/dm/syncer/opt_sharding_group_test.go +++ b/dm/syncer/opt_sharding_group_test.go @@ -73,6 +73,11 @@ func (s *optShardingGroupSuite) TestLowestFirstPosInOptGroups() { k.removeShardingReSync(&ShardingReSync{targetTable: utils.UnpackTableID(db2tbl)}) // should be pos11 now, pos21 is totally resolved require.Equal(s.T(), pos11.Position, k.lowestFirstLocationInGroups().Position) + + // reset + k.Reset() + require.Len(s.T(), k.groups, 0) + require.Len(s.T(), k.shardingReSyncs, 0) } func (s *optShardingGroupSuite) TestSync() { diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 634080e2fe0..8c70946b3fd 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -649,6 +649,7 @@ func (s *Syncer) reset() { s.sgk.ResetGroups() s.pessimist.Reset() case config.ShardOptimistic: + s.osgk.Reset() s.optimist.Reset() } } @@ -2886,8 +2887,18 @@ func (s *Syncer) trackOriginDDL(ev *replication.QueryEvent, ec eventContext) (ma return nil, err } - affectedTbls := make(map[string]map[string]struct{}) for _, sql := range qec.splitDDLs { + sqls, err := s.ddlWorker.processOneDDL(qec, sql) + if err != nil { + s.tctx.L().Warn("processOneDDL failed", zap.Error(err)) + qec.appliedDDLs = append(qec.appliedDDLs, sql) + } else { + qec.appliedDDLs = append(qec.appliedDDLs, sqls...) + } + } + + affectedTbls := make(map[string]map[string]struct{}) + for _, sql := range qec.appliedDDLs { ddlInfo, err := s.ddlWorker.genDDLInfo(qec, sql) if err != nil { return nil, err diff --git a/docs/swagger/docs.go b/docs/swagger/docs.go index c8cf4a220a8..2142d0c95de 100644 --- a/docs/swagger/docs.go +++ b/docs/swagger/docs.go @@ -2833,9 +2833,6 @@ var doc = `{ "consistent": { "$ref": "#/definitions/v2.ConsistentConfig" }, - "enable_old_value": { - "type": "boolean" - }, "enable_sync_point": { "type": "boolean" }, diff --git a/docs/swagger/swagger.json b/docs/swagger/swagger.json index b24edfb10f3..6ef5d9254c6 100644 --- a/docs/swagger/swagger.json +++ b/docs/swagger/swagger.json @@ -2814,9 +2814,6 @@ "consistent": { "$ref": "#/definitions/v2.ConsistentConfig" }, - "enable_old_value": { - "type": "boolean" - }, "enable_sync_point": { "type": "boolean" }, diff --git a/docs/swagger/swagger.yaml b/docs/swagger/swagger.yaml index 980dc8ee3b3..ba8053f5feb 100644 --- a/docs/swagger/swagger.yaml +++ b/docs/swagger/swagger.yaml @@ -1035,8 +1035,6 @@ definitions: type: boolean consistent: $ref: '#/definitions/v2.ConsistentConfig' - enable_old_value: - type: boolean enable_sync_point: type: boolean filter: diff --git a/metrics/grafana/TiCDC-Monitor-Summary.json b/metrics/grafana/TiCDC-Monitor-Summary.json index 4740f483fa5..f046cc34218 100644 --- a/metrics/grafana/TiCDC-Monitor-Summary.json +++ b/metrics/grafana/TiCDC-Monitor-Summary.json @@ -709,7 +709,7 @@ }, { "datasource": "${DS_TEST-CLUSTER}", - "description": "The status of each changefeed.\n\n0: Normal\n\n1: Error\n\n2: Failed\n\n3: Stopped\n\n4: Finished\n\n-1: Unknown", + "description": "The status of each changefeed.\n\n0: Normal\n\n1 and 6: Warning\n\n2: Failed\n\n3: Stopped\n\n4: Finished\n\n-1: Unknown", "fieldConfig": { "defaults": { "color": { @@ -748,7 +748,7 @@ { "from": "", "id": 2, - "text": "Error", + "text": "Warning", "to": "", "type": 1, "value": "1" @@ -780,22 +780,30 @@ { "from": "", "id": 6, + "text": "Warning", + "to": "", + "type": 1, + "value": "6" + }, + { + "from": "", + "id": 7, "text": "Unknown", "to": "", "type": 1, "value": "-1" }, { - "from": "5", - "id": 7, + "from": "7", + "id": 8, "text": "Other", "to": "10000", "type": 1, - "value": "5" + "value": "7" }, { - "from": "6", - "id": 8, + "from": "7", + "id": 9, "text": "-", "to": "1000", "type": 2 diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index 1cfe3758715..76c50e1a121 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -131,7 +131,7 @@ "gnetId": null, "graphTooltip": 1, "id": null, - "iteration": 1692166839167, + "iteration": 1693471596912, "links": [], "panels": [ { @@ -3900,7 +3900,7 @@ }, { "datasource": "${DS_TEST-CLUSTER}", - "description": "The status of each changefeed.\n\n0: Normal\n\n1: Error\n\n2: Failed\n\n3: Stopped\n\n4: Finished\n\n-1: Unknown", + "description": "The status of each changefeed.\n\n0: Normal\n\n1 and 6: Warning\n\n2: Failed\n\n3: Stopped\n\n4: Finished\n\n-1: Unknown", "fieldConfig": { "defaults": { "color": { @@ -3939,7 +3939,7 @@ { "from": "", "id": 2, - "text": "Error", + "text": "Warning", "to": "", "type": 1, "value": "1" @@ -8415,6 +8415,14 @@ "legendFormat": "{{namespace}}-{{changefeed}}-{{instance}}-P999", "queryType": "randomWalk", "refId": "A" + }, + { + "exemplar": true, + "expr": "rate(ticdc_sink_mq_claim_check_send_message_duration_sum[30s]) / rate(ticdc_sink_mq_claim_check_send_message_duration_count[30s])", + "hide": false, + "interval": "", + "legendFormat": "{{namespace}}-{{changefeed}}-{{instance}}-avg", + "refId": "B" } ], "thresholds": [], @@ -10940,7 +10948,7 @@ "h": 6, "w": 8, "x": 0, - "y": 11 + "y": 121 }, "hiddenSeries": false, "id": 271, @@ -11034,7 +11042,7 @@ "h": 6, "w": 8, "x": 8, - "y": 11 + "y": 121 }, "hiddenSeries": false, "id": 273, @@ -11142,7 +11150,7 @@ "h": 6, "w": 8, "x": 16, - "y": 11 + "y": 121 }, "hiddenSeries": false, "id": 272, @@ -11246,7 +11254,7 @@ "h": 6, "w": 8, "x": 0, - "y": 17 + "y": 127 }, "heatmap": {}, "hideZeroBuckets": true, @@ -11322,7 +11330,7 @@ "h": 6, "w": 8, "x": 8, - "y": 17 + "y": 127 }, "hiddenSeries": false, "id": 288, @@ -11450,7 +11458,7 @@ "h": 6, "w": 8, "x": 16, - "y": 17 + "y": 127 }, "hiddenSeries": false, "id": 286, @@ -11562,7 +11570,7 @@ "h": 6, "w": 8, "x": 0, - "y": 23 + "y": 133 }, "heatmap": {}, "hideZeroBuckets": true, @@ -11638,7 +11646,7 @@ "h": 6, "w": 8, "x": 8, - "y": 23 + "y": 133 }, "hiddenSeries": false, "id": 279, @@ -11741,7 +11749,7 @@ "h": 6, "w": 8, "x": 16, - "y": 23 + "y": 133 }, "hiddenSeries": false, "id": 275, @@ -11864,7 +11872,7 @@ "h": 6, "w": 8, "x": 0, - "y": 29 + "y": 139 }, "heatmap": {}, "hideZeroBuckets": true, @@ -11948,7 +11956,7 @@ "h": 6, "w": 8, "x": 8, - "y": 29 + "y": 139 }, "heatmap": {}, "hideZeroBuckets": true, @@ -12032,7 +12040,7 @@ "h": 6, "w": 8, "x": 16, - "y": 29 + "y": 139 }, "heatmap": {}, "hideZeroBuckets": true, @@ -12108,7 +12116,7 @@ "h": 6, "w": 8, "x": 0, - "y": 35 + "y": 145 }, "hiddenSeries": false, "id": 280, @@ -12208,7 +12216,7 @@ "h": 6, "w": 8, "x": 8, - "y": 35 + "y": 145 }, "hiddenSeries": false, "id": 283, @@ -12308,7 +12316,7 @@ "h": 6, "w": 8, "x": 16, - "y": 35 + "y": 145 }, "hiddenSeries": false, "id": 287, @@ -12416,7 +12424,7 @@ "h": 6, "w": 8, "x": 0, - "y": 41 + "y": 151 }, "heatmap": {}, "hideZeroBuckets": true, @@ -20053,5 +20061,5 @@ "timezone": "browser", "title": "${DS_TEST-CLUSTER}-TiCDC", "uid": "YiGL8hBZ1", - "version": 54 + "version": 55 } diff --git a/pkg/applier/redo_test.go b/pkg/applier/redo_test.go index 3c3c5fd3a56..30a1e9404c2 100644 --- a/pkg/applier/redo_test.go +++ b/pkg/applier/redo_test.go @@ -241,6 +241,10 @@ func getMockDB(t *testing.T) *sql.DB { Number: 1305, Message: "FUNCTION test.tidb_version does not exist", }) + mock.ExpectQuery("select tidb_version()").WillReturnError(&mysql.MySQLError{ + Number: 1305, + Message: "FUNCTION test.tidb_version does not exist", + }) mock.ExpectBegin() mock.ExpectExec("USE `test`;").WillReturnResult(sqlmock.NewResult(1, 1)) diff --git a/pkg/cmd/cli/cli_changefeed_create.go b/pkg/cmd/cli/cli_changefeed_create.go index ff85586841a..0a7b427adda 100644 --- a/pkg/cmd/cli/cli_changefeed_create.go +++ b/pkg/cmd/cli/cli_changefeed_create.go @@ -133,19 +133,17 @@ func (o *createChangefeedOptions) addFlags(cmd *cobra.Command) { } // complete adapts from the command line args to the data and client required. -func (o *createChangefeedOptions) complete(f factory.Factory, cmd *cobra.Command) error { +func (o *createChangefeedOptions) complete(f factory.Factory) error { client, err := f.APIV2Client() if err != nil { return err } o.apiClient = client - return o.completeReplicaCfg(cmd) + return o.completeReplicaCfg() } // completeCfg complete the replica config from file and cmd flags. -func (o *createChangefeedOptions) completeReplicaCfg( - cmd *cobra.Command, -) error { +func (o *createChangefeedOptions) completeReplicaCfg() error { cfg := config.GetDefaultReplicaConfig() if len(o.commonChangefeedOptions.configFile) > 0 { if err := o.commonChangefeedOptions.strictDecodeConfig("TiCDC changefeed", cfg); err != nil { @@ -163,17 +161,6 @@ func (o *createChangefeedOptions) completeReplicaCfg( return err } - for _, rules := range cfg.Sink.DispatchRules { - switch strings.ToLower(rules.PartitionRule) { - case "rowid", "index-value": - if cfg.EnableOldValue { - cmd.Printf("[WARN] This index-value distribution mode "+ - "does not guarantee row-level orderliness when "+ - "switching on the old value, so please use caution! dispatch-rules: %#v", rules) - } - } - } - if o.commonChangefeedOptions.schemaRegistry != "" { cfg.Sink.SchemaRegistry = putil.AddressOf(o.commonChangefeedOptions.schemaRegistry) } @@ -368,7 +355,7 @@ func newCmdCreateChangefeed(f factory.Factory) *cobra.Command { Run: func(cmd *cobra.Command, args []string) { ctx := cmdcontext.GetDefaultContext() - util.CheckErr(o.complete(f, cmd)) + util.CheckErr(o.complete(f)) util.CheckErr(o.validate(cmd)) util.CheckErr(o.run(ctx, cmd)) }, diff --git a/pkg/cmd/cli/cli_changefeed_create_test.go b/pkg/cmd/cli/cli_changefeed_create_test.go index 5b2667bf25d..427cbcef5f9 100644 --- a/pkg/cmd/cli/cli_changefeed_create_test.go +++ b/pkg/cmd/cli/cli_changefeed_create_test.go @@ -24,7 +24,6 @@ import ( v2 "github.com/pingcap/tiflow/cdc/api/v2" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" - cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/spf13/cobra" "github.com/stretchr/testify/require" ) @@ -133,7 +132,7 @@ func TestChangefeedCreateCli(t *testing.T) { cmd := newCmdCreateChangefeed(f) dir := t.TempDir() configPath := filepath.Join(dir, "cf.toml") - err := os.WriteFile(configPath, []byte("enable-old-value=false\r\nenable-sync-point=true\r\nsync-point-interval='20m'"), 0o644) + err := os.WriteFile(configPath, []byte("enable-sync-point=true\r\nsync-point-interval='20m'"), 0o644) require.Nil(t, err) os.Args = []string{ "create", @@ -172,41 +171,6 @@ func TestChangefeedCreateCli(t *testing.T) { cmd = newCmdCreateChangefeed(f) o := newCreateChangefeedOptions(newChangefeedCommonOptions()) o.commonChangefeedOptions.sortDir = "/tmp/test" - require.NoError(t, o.complete(f, cmd)) + require.NoError(t, o.complete(f)) require.Contains(t, o.validate(cmd).Error(), "creating changefeed with `--sort-dir`") } - -func TestChangefeedCreateCliAdjustEnableOldValue(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - f := newMockFactory(ctrl) - - // enable old value, but use avro as the encoding protocol, should be set to false. - dir := t.TempDir() - configPath := filepath.Join(dir, "adjust-old-value.toml") - err := os.WriteFile(configPath, []byte("enable-old-value=true"), 0o644) - require.NoError(t, err) - - cmd := new(cobra.Command) - o := newChangefeedCommonOptions() - o.addFlags(cmd) - - require.NoError(t, cmd.ParseFlags([]string{fmt.Sprintf("--config=%s", configPath)})) - require.NoError(t, cmd.ParseFlags([]string{"--sink-uri=kafka://127.0.0.1:9092/test?protocol=avro"})) - - opt := newCreateChangefeedOptions(o) - require.NoError(t, opt.complete(f, cmd)) - require.False(t, opt.cfg.EnableOldValue) - - // also enable the force replicate, should return error - configPath = filepath.Join(dir, "enable-old-value-force-replicate.toml") - err = os.WriteFile(configPath, []byte("enable-old-value=true\r\nforce-replicate = true"), 0o644) - require.NoError(t, err) - - require.NoError(t, cmd.ParseFlags([]string{"--sink-uri=kafka://127.0.0.1:9092/test?protocol=avro"})) - require.NoError(t, cmd.ParseFlags([]string{fmt.Sprintf("--config=%s", configPath)})) - - opt = newCreateChangefeedOptions(o) - err = opt.complete(f, cmd) - require.Error(t, cerror.ErrOldValueNotEnabled, err) -} diff --git a/pkg/cmd/server/server.go b/pkg/cmd/server/server.go index 113bd607dbc..df4c7a5c89d 100644 --- a/pkg/cmd/server/server.go +++ b/pkg/cmd/server/server.go @@ -131,8 +131,7 @@ func (o *options) run(cmd *cobra.Command) error { return errors.Trace(err) } // Drain the server before shutdown. - shutdownNotify := func() <-chan struct{} { return server.Drain() } - util.InitSignalHandling(shutdownNotify, cancel) + util.InitSignalHandling(server.Drain, cancel) // Run TiCDC server. err = server.Run(ctx) diff --git a/pkg/cmd/server/server_test.go b/pkg/cmd/server/server_test.go index bf054125101..678a868a6a4 100644 --- a/pkg/cmd/server/server_test.go +++ b/pkg/cmd/server/server_test.go @@ -151,9 +151,8 @@ func TestParseCfg(t *testing.T) { OwnerFlushInterval: config.TomlDuration(150 * time.Millisecond), ProcessorFlushInterval: config.TomlDuration(150 * time.Millisecond), Sorter: &config.SorterConfig{ - SortDir: config.DefaultSortDir, - CacheSizeInMB: 128, - MaxMemoryPercentage: 10, + SortDir: config.DefaultSortDir, + CacheSizeInMB: 128, }, Security: &config.SecurityConfig{ CertPath: "bb", @@ -171,18 +170,13 @@ func TestParseCfg(t *testing.T) { }, Debug: &config.DebugConfig{ DB: &config.DBConfig{ - Count: 8, - Concurrency: 128, - MaxOpenFiles: 10000, - BlockSize: 65536, - WriterBufferSize: 8388608, - Compression: "snappy", - WriteL0PauseTrigger: math.MaxInt32, - CompactionL0Trigger: 160, - CompactionDeletionThreshold: 10485760, - CompactionPeriod: 1800, - IteratorMaxAliveDuration: 10000, - IteratorSlowReadDuration: 256, + Count: 8, + MaxOpenFiles: 10000, + BlockSize: 65536, + WriterBufferSize: 8388608, + Compression: "snappy", + WriteL0PauseTrigger: math.MaxInt32, + CompactionL0Trigger: 160, }, // We expect the default configuration here. Messages: &config.MessagesConfig{ @@ -237,7 +231,6 @@ max-backups = 1 [sorter] sort-dir = "/tmp/just_a_test" cache-size-in-mb = 8 -max-memory-percentage = 3 [kv-client] region-retry-duration = "3s" @@ -245,7 +238,6 @@ region-retry-duration = "3s" [debug] [debug.db] count = 5 -concurrency = 6 max-open-files = 7 block-size = 32768 # 32 KB block-cache-size = 8 @@ -253,9 +245,6 @@ writer-buffer-size = 9 compression = "none" target-file-size-base = 10 compaction-l0-trigger = 11 -compaction-deletion-threshold = 15 -compaction-period = 16 -write-l0-slowdown-trigger = 12 write-l0-pause-trigger = 13 [debug.messages] @@ -306,9 +295,8 @@ check-balance-interval = "10s" OwnerFlushInterval: config.TomlDuration(600 * time.Millisecond), ProcessorFlushInterval: config.TomlDuration(600 * time.Millisecond), Sorter: &config.SorterConfig{ - SortDir: config.DefaultSortDir, - CacheSizeInMB: 8, - MaxMemoryPercentage: 3, + SortDir: config.DefaultSortDir, + CacheSizeInMB: 8, }, Security: &config.SecurityConfig{}, KVClient: &config.KVClientConfig{ @@ -322,18 +310,13 @@ check-balance-interval = "10s" }, Debug: &config.DebugConfig{ DB: &config.DBConfig{ - Count: 5, - Concurrency: 6, - MaxOpenFiles: 7, - BlockSize: 32768, - WriterBufferSize: 9, - Compression: "none", - CompactionL0Trigger: 11, - WriteL0PauseTrigger: 13, - IteratorMaxAliveDuration: 10000, - IteratorSlowReadDuration: 256, - CompactionDeletionThreshold: 15, - CompactionPeriod: 16, + Count: 5, + MaxOpenFiles: 7, + BlockSize: 32768, + WriterBufferSize: 9, + Compression: "none", + CompactionL0Trigger: 11, + WriteL0PauseTrigger: 13, }, Messages: &config.MessagesConfig{ ClientMaxBatchInterval: config.TomlDuration(500 * time.Millisecond), @@ -387,7 +370,6 @@ max-backups = 1 [sorter] sort-dir = "/tmp/just_a_test" cache-size-in-mb = 8 -max-memory-percentage = 3 [security] ca-path = "aa" @@ -439,9 +421,8 @@ cert-allowed-cn = ["dd","ee"] OwnerFlushInterval: config.TomlDuration(150 * time.Millisecond), ProcessorFlushInterval: config.TomlDuration(150 * time.Millisecond), Sorter: &config.SorterConfig{ - SortDir: config.DefaultSortDir, - CacheSizeInMB: 8, - MaxMemoryPercentage: 3, + SortDir: config.DefaultSortDir, + CacheSizeInMB: 8, }, Security: &config.SecurityConfig{ CertPath: "bb", @@ -459,18 +440,13 @@ cert-allowed-cn = ["dd","ee"] }, Debug: &config.DebugConfig{ DB: &config.DBConfig{ - Count: 8, - Concurrency: 128, - MaxOpenFiles: 10000, - BlockSize: 65536, - WriterBufferSize: 8388608, - Compression: "snappy", - WriteL0PauseTrigger: math.MaxInt32, - CompactionL0Trigger: 160, - CompactionDeletionThreshold: 10485760, - CompactionPeriod: 1800, - IteratorMaxAliveDuration: 10000, - IteratorSlowReadDuration: 256, + Count: 8, + MaxOpenFiles: 10000, + BlockSize: 65536, + WriterBufferSize: 8388608, + Compression: "snappy", + WriteL0PauseTrigger: math.MaxInt32, + CompactionL0Trigger: 160, }, // We expect the default configuration here. Messages: &config.MessagesConfig{ @@ -522,18 +498,13 @@ unknown3 = 3 require.Nil(t, err) require.Equal(t, &config.DebugConfig{ DB: &config.DBConfig{ - Count: 8, - Concurrency: 128, - MaxOpenFiles: 10000, - BlockSize: 65536, - WriterBufferSize: 8388608, - Compression: "snappy", - WriteL0PauseTrigger: math.MaxInt32, - CompactionL0Trigger: 160, - CompactionDeletionThreshold: 10485760, - CompactionPeriod: 1800, - IteratorMaxAliveDuration: 10000, - IteratorSlowReadDuration: 256, + Count: 8, + MaxOpenFiles: 10000, + BlockSize: 65536, + WriterBufferSize: 8388608, + Compression: "snappy", + WriteL0PauseTrigger: math.MaxInt32, + CompactionL0Trigger: 160, }, // We expect the default configuration here. Messages: &config.MessagesConfig{ diff --git a/pkg/cmd/util/helper_test.go b/pkg/cmd/util/helper_test.go index 07149f7cf03..3fdb15544f2 100644 --- a/pkg/cmd/util/helper_test.go +++ b/pkg/cmd/util/helper_test.go @@ -191,7 +191,7 @@ func TestAndWriteExampleReplicaTOML(t *testing.T) { err = cfg.ValidateAndAdjust(sinkURL) require.NoError(t, err) require.Equal(t, &config.SinkConfig{ - EncoderConcurrency: util.AddressOf(16), + EncoderConcurrency: util.AddressOf(config.DefaultEncoderGroupConcurrency), DispatchRules: []*config.DispatchRule{ {PartitionRule: "ts", TopicRule: "hello_{schema}", Matcher: []string{"test1.*", "test2.*"}}, {PartitionRule: "rowid", TopicRule: "{schema}_world", Matcher: []string{"test3.*", "test4.*"}}, @@ -230,7 +230,7 @@ func TestAndWriteStorageSinkTOML(t *testing.T) { require.NoError(t, err) require.Equal(t, &config.SinkConfig{ Protocol: util.AddressOf(config.ProtocolCanalJSON.String()), - EncoderConcurrency: util.AddressOf(16), + EncoderConcurrency: util.AddressOf(config.DefaultEncoderGroupConcurrency), Terminator: util.AddressOf(config.CRLF), TxnAtomicity: util.AddressOf(config.AtomicityLevel("")), DateSeparator: util.AddressOf("day"), diff --git a/pkg/compression/compress.go b/pkg/compression/compress.go index 72510a5e058..2704e683868 100644 --- a/pkg/compression/compress.go +++ b/pkg/compression/compress.go @@ -61,7 +61,7 @@ func Encode(cc string, data []byte) ([]byte, error) { default: } - return nil, cerror.ErrCompressionFailed.GenWithStack("Unsupported compression %d", cc) + return nil, cerror.ErrCompressionFailed.GenWithStack("Unsupported compression %s", cc) } // Decode the given data by the given compression codec. @@ -81,5 +81,5 @@ func Decode(cc string, data []byte) ([]byte, error) { default: } - return nil, cerror.ErrCompressionFailed.GenWithStack("Unsupported compression %d", cc) + return nil, cerror.ErrCompressionFailed.GenWithStack("Unsupported compression %s", cc) } diff --git a/pkg/config/config_test_data.go b/pkg/config/config_test_data.go index 5c0d3354dfd..0a79cc042ba 100644 --- a/pkg/config/config_test_data.go +++ b/pkg/config/config_test_data.go @@ -17,7 +17,6 @@ const ( testCfgTestReplicaConfigOutDated = `{ "memory-quota": 1073741824, "case-sensitive": false, - "enable-old-value": true, "force-replicate": true, "ignore-ineligible-table":false, "check-gc-safe-point": true, @@ -35,7 +34,7 @@ const ( "worker-num": 3 }, "sink": { - "encoder-concurrency": 16, + "encoder-concurrency": 32, "terminator": "\r\n", "date-separator": "day", "dispatch-rules": [ @@ -106,7 +105,7 @@ const ( "sorter": { "sort-dir": "/tmp/sorter", "cache-size-in-mb": 128, - "max-memory-percentage": 10, + "max-memory-percentage": 0, "max-memory-consumption": 0, "num-workerpool-goroutine": 0, "num-concurrent-worker": 0, @@ -131,17 +130,12 @@ const ( "debug": { "db": { "count": 8, - "concurrency": 128, "max-open-files": 10000, "block-size": 65536, "writer-buffer-size": 8388608, "compression": "snappy", "write-l0-pause-trigger": 2147483647, - "compaction-l0-trigger": 160, - "compaction-deletion-threshold": 10485760, - "compaction-period": 1800, - "iterator-max-alive-duration": 10000, - "iterator-slow-read-duration": 256 + "compaction-l0-trigger": 160 }, "messages": { "client-max-batch-interval": 10000000, @@ -170,7 +164,6 @@ const ( testCfgTestReplicaConfigMarshal1 = `{ "memory-quota": 1073741824, "case-sensitive": false, - "enable-old-value": true, "force-replicate": true, "ignore-ineligible-table":false, "check-gc-safe-point": true, @@ -189,7 +182,7 @@ const ( "worker-num": 3 }, "sink": { - "encoder-concurrency": 16, + "encoder-concurrency": 32, "protocol": "open-protocol", "column-selectors": [ { @@ -251,18 +244,13 @@ const ( "avro-decimal-handling-mode": "string", "avro-bigint-unsigned-handling-mode": "string" }, - "large-message-handle": { - "large-message-handle-option": "handle-key-only", - "claim-check-storage-uri": "", - "claim-check-compression": "" - }, "large-message-handle": { "large-message-handle-option": "handle-key-only", "large-message-handle-compression": "", "claim-check-storage-uri": "" }, "glue-schema-registry-config": { - "region":"region", + "region":"region", "registry-name":"registry" } }, @@ -322,7 +310,6 @@ const ( testCfgTestReplicaConfigMarshal2 = `{ "memory-quota": 1073741824, "case-sensitive": false, - "enable-old-value": true, "force-replicate": true, "ignore-ineligible-table":false, "check-gc-safe-point": true, @@ -340,7 +327,7 @@ const ( "worker-num": 3 }, "sink": { - "encoder-concurrency": 16, + "encoder-concurrency": 32, "dispatchers": null, "protocol": "open-protocol", "column-selectors": [ @@ -409,7 +396,7 @@ const ( "claim-check-compression": "" }, "glue-schema-registry-config": { - "region":"region", + "region":"region", "registry-name":"registry" } }, diff --git a/pkg/config/consistent.go b/pkg/config/consistent.go index 85baaf5673f..636edcf865f 100644 --- a/pkg/config/consistent.go +++ b/pkg/config/consistent.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tidb/br/pkg/storage" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/redo" + "github.com/pingcap/tiflow/pkg/util" ) // ConsistentConfig represents replication consistency config for a changefeed. @@ -56,3 +57,8 @@ func (c *ConsistentConfig) ValidateAndAdjust() error { } return redo.ValidateStorage(uri) } + +// MaskSensitiveData masks sensitive data in ConsistentConfig +func (c *ConsistentConfig) MaskSensitiveData() { + c.Storage = util.MaskSensitiveDataInURI(c.Storage) +} diff --git a/pkg/config/db.go b/pkg/config/db.go index eca6103cdd9..5af0f29fb1c 100644 --- a/pkg/config/db.go +++ b/pkg/config/db.go @@ -21,10 +21,6 @@ type DBConfig struct { // // The default value is 8. Count int `toml:"count" json:"count"` - // Concurrency is the maximum write and read concurrency. - // - // The default value is 256. - Concurrency int `toml:"concurrency" json:"concurrency"` // MaxOpenFiles is the maximum number of open FD by db sorter. // // The default value is 10000. @@ -53,28 +49,6 @@ type DBConfig struct { // // The default value is 160. CompactionL0Trigger int `toml:"compaction-l0-trigger" json:"compaction-l0-trigger"` - // CompactionDeletionThreshold defines the threshold of the number of deletion that - // trigger compaction. - // - // The default value is 10 * 1024 * 1024, 10485760. - // Assume every key-value is about 1KB, 10485760 is about deleting 10GB data. - CompactionDeletionThreshold int `toml:"compaction-deletion-threshold" json:"compaction-deletion-threshold"` - // CompactionDeletionThreshold defines the threshold of the number of deletion that - // trigger compaction. - // - // The default value is 30 minutes, 1800. - CompactionPeriod int `toml:"compaction-period" json:"compaction-period"` - - // IteratorMaxAliveDuration the maximum iterator alive duration in ms. - // - // The default value is 10000, 10s - IteratorMaxAliveDuration int `toml:"iterator-max-alive-duration" json:"iterator-max-alive-duration"` - - // IteratorSlowReadDuration is the iterator slow read threshold. - // A reading that exceeds the duration triggers a db compaction. - // - // The default value is 256, 256ms. - IteratorSlowReadDuration int `toml:"iterator-slow-read-duration" json:"iterator-slow-read-duration"` } // ValidateAndAdjust validates and adjusts the db configuration diff --git a/pkg/config/large_message.go b/pkg/config/large_message.go index d56e7691ea9..83d6d255853 100644 --- a/pkg/config/large_message.go +++ b/pkg/config/large_message.go @@ -42,16 +42,21 @@ func NewDefaultLargeMessageHandleConfig() *LargeMessageHandleConfig { } } -// Validate the Config. -func (c *LargeMessageHandleConfig) Validate(protocol Protocol, enableTiDBExtension bool) error { - // compression can be enabled independently - if c.LargeMessageHandleCompression != "" { - if !compression.Supported(c.LargeMessageHandleCompression) { - return cerror.ErrInvalidReplicaConfig.GenWithStack( - "large message handle compression is not supported, got %s", c.LargeMessageHandleCompression) - } +// AdjustAndValidate the Config. +func (c *LargeMessageHandleConfig) AdjustAndValidate(protocol Protocol, enableTiDBExtension bool) error { + if c.LargeMessageHandleOption == "" { + c.LargeMessageHandleOption = LargeMessageHandleOptionNone + } + + if c.LargeMessageHandleCompression == "" { + c.LargeMessageHandleCompression = compression.None } + // compression can be enabled independently + if !compression.Supported(c.LargeMessageHandleCompression) { + return cerror.ErrInvalidReplicaConfig.GenWithStack( + "large message handle compression is not supported, got %s", c.LargeMessageHandleCompression) + } if c.LargeMessageHandleOption == LargeMessageHandleOptionNone { return nil } diff --git a/pkg/config/large_message_test.go b/pkg/config/large_message_test.go index e06d5717632..3bc1dd6b212 100644 --- a/pkg/config/large_message_test.go +++ b/pkg/config/large_message_test.go @@ -29,19 +29,19 @@ func TestLargeMessageHandle4Compression(t *testing.T) { // unsupported compression, return error largeMessageHandle.LargeMessageHandleCompression = "zstd" - err := largeMessageHandle.Validate(ProtocolCanalJSON, false) + err := largeMessageHandle.AdjustAndValidate(ProtocolCanalJSON, false) require.ErrorIs(t, err, cerror.ErrInvalidReplicaConfig) largeMessageHandle.LargeMessageHandleCompression = compression.LZ4 - err = largeMessageHandle.Validate(ProtocolCanalJSON, false) + err = largeMessageHandle.AdjustAndValidate(ProtocolCanalJSON, false) require.NoError(t, err) largeMessageHandle.LargeMessageHandleCompression = compression.Snappy - err = largeMessageHandle.Validate(ProtocolCanalJSON, false) + err = largeMessageHandle.AdjustAndValidate(ProtocolCanalJSON, false) require.NoError(t, err) largeMessageHandle.LargeMessageHandleCompression = compression.None - err = largeMessageHandle.Validate(ProtocolCanalJSON, false) + err = largeMessageHandle.AdjustAndValidate(ProtocolCanalJSON, false) require.NoError(t, err) } @@ -50,11 +50,11 @@ func TestLargeMessageHandle4NotSupportedProtocol(t *testing.T) { largeMessageHandle := NewDefaultLargeMessageHandleConfig() - err := largeMessageHandle.Validate(ProtocolCanal, true) + err := largeMessageHandle.AdjustAndValidate(ProtocolCanal, true) require.NoError(t, err) largeMessageHandle.LargeMessageHandleOption = LargeMessageHandleOptionHandleKeyOnly - err = largeMessageHandle.Validate(ProtocolCanal, true) + err = largeMessageHandle.AdjustAndValidate(ProtocolCanal, true) require.ErrorIs(t, err, cerror.ErrInvalidReplicaConfig) } @@ -64,7 +64,7 @@ func TestLargeMessageHandle4CanalJSON(t *testing.T) { // large-message-handle not set, always no error largeMessageHandle := NewDefaultLargeMessageHandleConfig() - err := largeMessageHandle.Validate(ProtocolCanalJSON, false) + err := largeMessageHandle.AdjustAndValidate(ProtocolCanalJSON, false) require.NoError(t, err) require.True(t, largeMessageHandle.Disabled()) @@ -78,11 +78,11 @@ func TestLargeMessageHandle4CanalJSON(t *testing.T) { } // `enable-tidb-extension` is false, return error - err := largeMessageHandle.Validate(ProtocolCanalJSON, false) + err := largeMessageHandle.AdjustAndValidate(ProtocolCanalJSON, false) require.ErrorIs(t, err, cerror.ErrInvalidReplicaConfig) // `enable-tidb-extension` is true, no error - err = largeMessageHandle.Validate(ProtocolCanalJSON, true) + err = largeMessageHandle.AdjustAndValidate(ProtocolCanalJSON, true) require.NoError(t, err) require.Equal(t, option, largeMessageHandle.LargeMessageHandleOption) } @@ -94,7 +94,7 @@ func TestLargeMessageHandle4OpenProtocol(t *testing.T) { // large-message-handle not set, always no error largeMessageHandle := NewDefaultLargeMessageHandleConfig() - err := largeMessageHandle.Validate(ProtocolOpen, false) + err := largeMessageHandle.AdjustAndValidate(ProtocolOpen, false) require.NoError(t, err) require.True(t, largeMessageHandle.Disabled()) @@ -108,11 +108,11 @@ func TestLargeMessageHandle4OpenProtocol(t *testing.T) { } // `enable-tidb-extension` is false, return error - err := largeMessageHandle.Validate(ProtocolOpen, false) + err := largeMessageHandle.AdjustAndValidate(ProtocolOpen, false) require.NoError(t, err) // `enable-tidb-extension` is true, no error - err = largeMessageHandle.Validate(ProtocolOpen, true) + err = largeMessageHandle.AdjustAndValidate(ProtocolOpen, true) require.NoError(t, err) require.Equal(t, o, largeMessageHandle.LargeMessageHandleOption) diff --git a/pkg/config/replica_config.go b/pkg/config/replica_config.go index 6ccfa25b36c..18bfc6825a8 100644 --- a/pkg/config/replica_config.go +++ b/pkg/config/replica_config.go @@ -41,7 +41,6 @@ const ( var defaultReplicaConfig = &ReplicaConfig{ MemoryQuota: DefaultChangefeedMemoryQuota, CaseSensitive: true, - EnableOldValue: true, CheckGCSafePoint: true, EnableSyncPoint: util.AddressOf(false), SyncPointInterval: util.AddressOf(10 * time.Minute), @@ -60,7 +59,7 @@ var defaultReplicaConfig = &ReplicaConfig{ NullString: NULL, BinaryEncodingMethod: BinaryEncodingBase64, }, - EncoderConcurrency: util.AddressOf(16), + EncoderConcurrency: util.AddressOf(DefaultEncoderGroupConcurrency), Terminator: util.AddressOf(CRLF), DateSeparator: util.AddressOf(DateSeparatorDay.String()), EnablePartitionSeparator: util.AddressOf(true), @@ -111,7 +110,6 @@ type ReplicaConfig replicaConfig type replicaConfig struct { MemoryQuota uint64 `toml:"memory-quota" json:"memory-quota"` CaseSensitive bool `toml:"case-sensitive" json:"case-sensitive"` - EnableOldValue bool `toml:"enable-old-value" json:"enable-old-value"` ForceReplicate bool `toml:"force-replicate" json:"force-replicate"` CheckGCSafePoint bool `toml:"check-gc-safe-point" json:"check-gc-safe-point"` // EnableSyncPoint is only available when the downstream is a Database. @@ -201,11 +199,6 @@ func (c *ReplicaConfig) ValidateAndAdjust(sinkURI *url.URL) error { // check sin if err != nil { return err } - - err = c.adjustEnableOldValueAndVerifyForceReplicate(sinkURI) - if err != nil { - return err - } } if c.Consistent != nil { @@ -293,50 +286,12 @@ func isSinkCompatibleWithSpanReplication(u *url.URL) bool { (strings.Contains(u.Scheme, "kafka") || strings.Contains(u.Scheme, "blackhole")) } -// AdjustEnableOldValue adjust the old value configuration by the sink scheme and encoding protocol -func (c *ReplicaConfig) AdjustEnableOldValue(scheme, protocol string) { - if sink.IsMySQLCompatibleScheme(scheme) { - return - } - - if c.EnableOldValue { - _, ok := ForceDisableOldValueProtocols[protocol] - if ok { - log.Warn("Attempting to replicate with old value enabled, but the specified protocol must disable old value. "+ - "CDC will disable old value and continue.", zap.String("protocol", protocol)) - c.EnableOldValue = false - } - return - } - - _, ok := ForceEnableOldValueProtocols[protocol] - if ok { - log.Warn("Attempting to replicate with old value disabled, but the specified protocol must enable old value. "+ - "CDC will enable old value and continue.", zap.String("protocol", protocol)) - c.EnableOldValue = true - } -} - -func (c *ReplicaConfig) adjustEnableOldValueAndVerifyForceReplicate(sinkURI *url.URL) error { - scheme := strings.ToLower(sinkURI.Scheme) - protocol := sinkURI.Query().Get(ProtocolKey) - if protocol != "" { - c.Sink.Protocol = util.AddressOf(protocol) - } - protocol = util.GetOrZero(c.Sink.Protocol) - c.AdjustEnableOldValue(scheme, protocol) - - if !c.ForceReplicate { - return nil +// MaskSensitiveData masks sensitive data in ReplicaConfig +func (c *ReplicaConfig) MaskSensitiveData() { + if c.Sink != nil { + c.Sink.MaskSensitiveData() } - - // MySQL Sink require the old value feature must be enabled to allow delete event send to downstream. - if sink.IsMySQLCompatibleScheme(scheme) { - if !c.EnableOldValue { - log.Error("force replicate, old value feature is disabled for the changefeed using mysql sink") - return cerror.ErrIncompatibleConfig.GenWithStackByArgs() - } + if c.Consistent != nil { + c.Consistent.MaskSensitiveData() } - - return nil } diff --git a/pkg/config/replica_config_test.go b/pkg/config/replica_config_test.go index fbbeb22e818..dca4a3d11b2 100644 --- a/pkg/config/replica_config_test.go +++ b/pkg/config/replica_config_test.go @@ -21,7 +21,7 @@ import ( "time" "github.com/aws/aws-sdk-go/aws" - cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/compression" "github.com/pingcap/tiflow/pkg/integrity" "github.com/pingcap/tiflow/pkg/util" "github.com/stretchr/testify/require" @@ -194,15 +194,6 @@ func TestReplicaConfigValidate(t *testing.T) { require.NoError(t, err) require.NoError(t, conf.ValidateAndAdjust(sinkURL)) - // Incorrect sink configuration. - conf = GetDefaultReplicaConfig() - conf.Sink.Protocol = util.AddressOf("canal") - conf.EnableOldValue = false - - err = conf.ValidateAndAdjust(sinkURL) - require.NoError(t, err) - require.True(t, conf.EnableOldValue) - conf = GetDefaultReplicaConfig() conf.Sink.DispatchRules = []*DispatchRule{ {Matcher: []string{"a.b"}, DispatcherRule: "d1", PartitionRule: "r1"}, @@ -336,74 +327,59 @@ func TestIsSinkCompatibleWithSpanReplication(t *testing.T) { } } -func TestAdjustEnableOldValueAndVerifyForceReplicate(t *testing.T) { - t.Parallel() - - config := GetDefaultReplicaConfig() - config.EnableOldValue = false - - // mysql sink, do not adjust enable-old-value - sinkURI, err := url.Parse("mysql://") - require.NoError(t, err) - err = config.adjustEnableOldValueAndVerifyForceReplicate(sinkURI) - require.NoError(t, err) - require.False(t, config.EnableOldValue) - - // mysql sink, `enable-old-value` false, `force-replicate` true, should return error - config.ForceReplicate = true - err = config.adjustEnableOldValueAndVerifyForceReplicate(sinkURI) - require.Error(t, cerror.ErrOldValueNotEnabled, err) - - // canal, `enable-old-value` false, `force-replicate` false, no error, `enable-old-value` adjust to true - config.ForceReplicate = false - config.EnableOldValue = false - // canal require old value enabled - sinkURI, err = url.Parse("kafka://127.0.0.1:9092/test?protocol=canal") - require.NoError(t, err) - - err = config.adjustEnableOldValueAndVerifyForceReplicate(sinkURI) - require.NoError(t, err) - require.True(t, config.EnableOldValue) - - // canal, `force-replicate` true, `enable-old-value` true, no error - config.ForceReplicate = true - config.EnableOldValue = true - err = config.adjustEnableOldValueAndVerifyForceReplicate(sinkURI) - require.NoError(t, err) - require.True(t, config.ForceReplicate) - require.True(t, config.EnableOldValue) - - // avro, `enable-old-value` false, `force-replicate` false, no error - config.ForceReplicate = false - config.EnableOldValue = false - sinkURI, err = url.Parse("kafka://127.0.0.1:9092/test?protocol=avro") - require.NoError(t, err) - - err = config.adjustEnableOldValueAndVerifyForceReplicate(sinkURI) - require.NoError(t, err) - require.False(t, config.EnableOldValue) +func TestValidateAndAdjustLargeMessageHandle(t *testing.T) { + cfg := GetDefaultReplicaConfig() + cfg.Sink.KafkaConfig = &KafkaConfig{ + LargeMessageHandle: NewDefaultLargeMessageHandleConfig(), + } + cfg.Sink.KafkaConfig.LargeMessageHandle.LargeMessageHandleOption = "" + cfg.Sink.KafkaConfig.LargeMessageHandle.LargeMessageHandleCompression = "" - // avro, `enable-old-value` true, no error, set to false. no matter `force-replicate` - config.EnableOldValue = true - config.ForceReplicate = true - err = config.adjustEnableOldValueAndVerifyForceReplicate(sinkURI) + rawURL := "kafka://127.0.0.1:9092/canal-json-test?protocol=canal-json&enable-tidb-extension=true" + sinkURL, err := url.Parse(rawURL) require.NoError(t, err) - require.False(t, config.EnableOldValue) - // csv, `enable-old-value` false, `force-replicate` false, no error - config.EnableOldValue = false - config.ForceReplicate = false - sinkURI, err = url.Parse("s3://xxx/yyy?protocol=csv") + err = cfg.ValidateAndAdjust(sinkURL) require.NoError(t, err) - err = config.adjustEnableOldValueAndVerifyForceReplicate(sinkURI) - require.NoError(t, err) - require.False(t, config.EnableOldValue) + require.Equal(t, LargeMessageHandleOptionNone, cfg.Sink.KafkaConfig.LargeMessageHandle.LargeMessageHandleOption) + require.Equal(t, compression.None, cfg.Sink.KafkaConfig.LargeMessageHandle.LargeMessageHandleCompression) +} - // csv, `enable-old-value` true, no error, set to false. no matter `force-replicate` - config.EnableOldValue = true - config.ForceReplicate = true - err = config.adjustEnableOldValueAndVerifyForceReplicate(sinkURI) - require.NoError(t, err) - require.False(t, config.EnableOldValue) +func TestMaskSensitiveData(t *testing.T) { + config := ReplicaConfig{ + Sink: nil, + Consistent: nil, + } + config.MaskSensitiveData() + require.Nil(t, config.Sink) + require.Nil(t, config.Consistent) + config.Sink = &SinkConfig{} + config.Sink.KafkaConfig = &KafkaConfig{ + SASLOAuthTokenURL: aws.String("http://abc.com?password=bacd"), + SASLOAuthClientSecret: aws.String("bacd"), + SASLPassword: aws.String("bacd"), + SASLGssAPIPassword: aws.String("bacd"), + Key: aws.String("bacd"), + GlueSchemaRegistryConfig: &GlueSchemaRegistryConfig{ + AccessKey: "abc", + SecretAccessKey: "def", + Token: "aaa", + }, + } + config.Sink.SchemaRegistry = aws.String("http://abc.com?password=bacd") + config.Consistent = &ConsistentConfig{ + Storage: "http://abc.com?password=bacd", + } + config.MaskSensitiveData() + require.Equal(t, "http://abc.com?password=xxxxx", *config.Sink.SchemaRegistry) + require.Equal(t, "http://abc.com?password=xxxxx", config.Consistent.Storage) + require.Equal(t, "http://abc.com?password=xxxxx", *config.Sink.KafkaConfig.SASLOAuthTokenURL) + require.Equal(t, "******", *config.Sink.KafkaConfig.SASLOAuthClientSecret) + require.Equal(t, "******", *config.Sink.KafkaConfig.Key) + require.Equal(t, "******", *config.Sink.KafkaConfig.SASLPassword) + require.Equal(t, "******", *config.Sink.KafkaConfig.SASLGssAPIPassword) + require.Equal(t, "******", config.Sink.KafkaConfig.GlueSchemaRegistryConfig.SecretAccessKey) + require.Equal(t, "******", config.Sink.KafkaConfig.GlueSchemaRegistryConfig.Token) + require.Equal(t, "******", config.Sink.KafkaConfig.GlueSchemaRegistryConfig.AccessKey) } diff --git a/pkg/config/server_config.go b/pkg/config/server_config.go index a62c80e1183..feaff34fa03 100644 --- a/pkg/config/server_config.go +++ b/pkg/config/server_config.go @@ -106,9 +106,8 @@ var defaultServerConfig = &ServerConfig{ OwnerFlushInterval: TomlDuration(50 * time.Millisecond), ProcessorFlushInterval: TomlDuration(50 * time.Millisecond), Sorter: &SorterConfig{ - SortDir: DefaultSortDir, - CacheSizeInMB: 128, // By default use 128M memory as sorter cache. - MaxMemoryPercentage: 10, // Deprecated. + SortDir: DefaultSortDir, + CacheSizeInMB: 128, // By default, use 128M memory as sorter cache. }, Security: &SecurityConfig{}, KVClient: &KVClientConfig{ @@ -127,17 +126,12 @@ var defaultServerConfig = &ServerConfig{ Count: 8, // Following configs are optimized for write/read throughput. // Users should not change them. - Concurrency: 128, - MaxOpenFiles: 10000, - BlockSize: 65536, - WriterBufferSize: 8388608, - Compression: "snappy", - WriteL0PauseTrigger: math.MaxInt32, - CompactionL0Trigger: 160, - CompactionDeletionThreshold: 10485760, - CompactionPeriod: 1800, - IteratorMaxAliveDuration: 10000, - IteratorSlowReadDuration: 256, + MaxOpenFiles: 10000, + BlockSize: 65536, + WriterBufferSize: 8388608, + Compression: "snappy", + WriteL0PauseTrigger: math.MaxInt32, + CompactionL0Trigger: 160, }, Messages: defaultMessageConfig.Clone(), @@ -168,8 +162,7 @@ type ServerConfig struct { Sorter *SorterConfig `toml:"sorter" json:"sorter"` Security *SecurityConfig `toml:"security" json:"security"` - // DEPRECATED: after using pull based sink, this config is useless. - // Because we do not control the memory usage by table anymore. + // Deprecated: we don't use this field anymore. PerTableMemoryQuota uint64 `toml:"per-table-memory-quota" json:"per-table-memory-quota"` KVClient *KVClientConfig `toml:"kv-client" json:"kv-client"` Debug *DebugConfig `toml:"debug" json:"debug"` diff --git a/pkg/config/sink.go b/pkg/config/sink.go index 2476f84edb0..2d2c8db7b26 100644 --- a/pkg/config/sink.go +++ b/pkg/config/sink.go @@ -16,10 +16,12 @@ package config import ( "fmt" "net/url" + "strconv" "strings" "time" "github.com/apache/pulsar-client-go/pulsar" + "github.com/aws/aws-sdk-go-v2/aws" "github.com/pingcap/errors" "github.com/pingcap/log" cerror "github.com/pingcap/tiflow/pkg/errors" @@ -76,6 +78,9 @@ const ( // DefaultPulsarProducerCacheSize is the default size of the cache for producers // 10240 producers maybe cost 1.1G memory DefaultPulsarProducerCacheSize = 10240 + + // DefaultEncoderGroupConcurrency is the default concurrency of encoder group. + DefaultEncoderGroupConcurrency = 32 ) // AtomicityLevel represents the atomicity level of a changefeed. @@ -107,19 +112,6 @@ func (l AtomicityLevel) validate(scheme string) error { return nil } -// ForceEnableOldValueProtocols specifies which protocols need to be forced to enable old value. -var ForceEnableOldValueProtocols = map[string]struct{}{ - ProtocolCanal.String(): {}, - ProtocolCanalJSON.String(): {}, - ProtocolMaxwell.String(): {}, -} - -// ForceDisableOldValueProtocols specifies protocols need to be forced to disable old value. -var ForceDisableOldValueProtocols = map[string]struct{}{ - ProtocolAvro.String(): {}, - ProtocolCsv.String(): {}, -} - // SinkConfig represents sink config for a changefeed type SinkConfig struct { TxnAtomicity *AtomicityLevel `toml:"transaction-atomicity" json:"transaction-atomicity,omitempty"` @@ -172,6 +164,19 @@ type SinkConfig struct { AdvanceTimeoutInSec *uint `toml:"advance-timeout-in-sec" json:"advance-timeout-in-sec,omitempty"` } +// MaskSensitiveData masks sensitive data in SinkConfig +func (s *SinkConfig) MaskSensitiveData() { + if s.SchemaRegistry != nil { + s.SchemaRegistry = aws.String(util.MaskSensitiveDataInURI(*s.SchemaRegistry)) + } + if s.KafkaConfig != nil { + s.KafkaConfig.MaskSensitiveData() + } + if s.PulsarConfig != nil { + s.PulsarConfig.MaskSensitiveData() + } +} + // CSVConfig defines a series of configuration items for csv codec. type CSVConfig struct { // delimiter between fields @@ -340,6 +345,22 @@ type KafkaConfig struct { GlueSchemaRegistryConfig *GlueSchemaRegistryConfig `toml:"glue-schema-registry-config" json:"glue-schema-registry-config"` } +// MaskSensitiveData masks sensitive data in KafkaConfig +func (k *KafkaConfig) MaskSensitiveData() { + k.SASLPassword = aws.String("******") + k.SASLGssAPIPassword = aws.String("******") + k.SASLOAuthClientSecret = aws.String("******") + k.Key = aws.String("******") + if k.GlueSchemaRegistryConfig != nil { + k.GlueSchemaRegistryConfig.AccessKey = "******" + k.GlueSchemaRegistryConfig.Token = "******" + k.GlueSchemaRegistryConfig.SecretAccessKey = "******" + } + if k.SASLOAuthTokenURL != nil { + k.SASLOAuthTokenURL = aws.String(util.MaskSensitiveDataInURI(*k.SASLOAuthTokenURL)) + } +} + // PulsarCompressionType is the compression type for pulsar type PulsarCompressionType string @@ -476,12 +497,24 @@ type PulsarConfig struct { // and 'type' always use 'client_credentials' OAuth2 *OAuth2 `toml:"oauth2" json:"oauth2,omitempty"` - // Configure the service brokerUrl for the Pulsar service. - // This parameter from the sink-uri - brokerURL string `toml:"-" json:"-"` + // BrokerURL is used to configure service brokerUrl for the Pulsar service. + // This parameter is a part of the `sink-uri`. Internal use only. + BrokerURL string `toml:"-" json:"-"` + // SinkURI is the parsed sinkURI. Internal use only. + SinkURI *url.URL `toml:"-" json:"-"` +} - // parse the sinkURI - u *url.URL `toml:"-" json:"-"` +// MaskSensitiveData masks sensitive data in PulsarConfig +func (c *PulsarConfig) MaskSensitiveData() { + if c.AuthenticationToken != nil { + c.AuthenticationToken = aws.String("******") + } + if c.BasicPassword != nil { + c.BasicPassword = aws.String("******") + } + if c.OAuth2 != nil { + c.OAuth2.OAuth2PrivateKey = "******" + } } // Check get broker url @@ -498,29 +531,9 @@ func (c *PulsarConfig) validate() (err error) { return nil } -// GetBrokerURL get broker url -func (c *PulsarConfig) GetBrokerURL() string { - return c.brokerURL -} - -// SetBrokerURL get broker url -func (c *PulsarConfig) SetBrokerURL(brokerURL string) { - c.brokerURL = brokerURL -} - -// GetSinkURI get sink uri -func (c *PulsarConfig) GetSinkURI() *url.URL { - return c.u -} - -// SetSinkURI get sink uri -func (c *PulsarConfig) SetSinkURI(u *url.URL) { - c.u = u -} - // GetDefaultTopicName get default topic name func (c *PulsarConfig) GetDefaultTopicName() string { - topicName := c.u.Path + topicName := c.SinkURI.Path return topicName[1:] } @@ -553,6 +566,33 @@ type CloudStorageConfig struct { } func (s *SinkConfig) validateAndAdjust(sinkURI *url.URL) error { + if err := s.validateAndAdjustSinkURI(sinkURI); err != nil { + return err + } + + if sink.IsMySQLCompatibleScheme(sinkURI.Scheme) { + return nil + } + + protocol, _ := ParseSinkProtocolFromString(util.GetOrZero(s.Protocol)) + + if s.KafkaConfig != nil && s.KafkaConfig.LargeMessageHandle != nil { + var ( + enableTiDBExtension bool + err error + ) + if s := sinkURI.Query().Get("enable-tidb-extension"); s != "" { + enableTiDBExtension, err = strconv.ParseBool(s) + if err != nil { + return errors.Trace(err) + } + } + err = s.KafkaConfig.LargeMessageHandle.AdjustAndValidate(protocol, enableTiDBExtension) + if err != nil { + return err + } + } + if s.SchemaRegistry != nil && (s.KafkaConfig != nil && s.KafkaConfig.GlueSchemaRegistryConfig != nil) { return cerror.ErrInvalidReplicaConfig. @@ -561,6 +601,7 @@ func (s *SinkConfig) validateAndAdjust(sinkURI *url.URL) error { "schema-registry is used by confluent schema registry, " + "glue-schema-registry-config is used by aws glue schema registry") } + if s.KafkaConfig != nil && s.KafkaConfig.GlueSchemaRegistryConfig != nil { err := s.KafkaConfig.GlueSchemaRegistryConfig.Validate() if err != nil { @@ -568,12 +609,10 @@ func (s *SinkConfig) validateAndAdjust(sinkURI *url.URL) error { } } - if err := s.validateAndAdjustSinkURI(sinkURI); err != nil { - return err - } - - if sink.IsMySQLCompatibleScheme(sinkURI.Scheme) { - return nil + if sink.IsPulsarScheme(sinkURI.Scheme) && s.PulsarConfig == nil { + s.PulsarConfig = &PulsarConfig{ + SinkURI: sinkURI, + } } if s.PulsarConfig != nil { @@ -608,7 +647,6 @@ func (s *SinkConfig) validateAndAdjust(sinkURI *url.URL) error { s.Terminator = util.AddressOf(CRLF) } - protocol, _ := ParseSinkProtocolFromString(util.GetOrZero(s.Protocol)) if util.GetOrZero(s.DeleteOnlyOutputHandleKeyColumns) && protocol == ProtocolCsv { return cerror.ErrSinkInvalidConfig.GenWithStack( "CSV protocol always output all columns for the delete event, " + @@ -669,7 +707,7 @@ func (s *SinkConfig) validateAndAdjustSinkURI(sinkURI *url.URL) error { return err } - // Validate that protocol is compatible with the scheme. For testing purposes, + // Adjust that protocol is compatible with the scheme. For testing purposes, // any protocol should be legal for blackhole. if sink.IsMQScheme(sinkURI.Scheme) || sink.IsStorageScheme(sinkURI.Scheme) { _, err := ParseSinkProtocolFromString(util.GetOrZero(s.Protocol)) diff --git a/pkg/config/sorter.go b/pkg/config/sorter.go index 2310dc2fa07..9b740593000 100644 --- a/pkg/config/sorter.go +++ b/pkg/config/sorter.go @@ -27,21 +27,15 @@ type SorterConfig struct { // Cache size of sorter in MB. CacheSizeInMB uint64 `toml:"cache-size-in-mb" json:"cache-size-in-mb"` - // the maximum memory use percentage that allows in-memory sorting - // Deprecated: use CacheSizeInMB instead. + // Deprecated: we don't use this field anymore. MaxMemoryPercentage int `toml:"max-memory-percentage" json:"max-memory-percentage"` - - // the maximum memory consumption allowed for in-memory sorting - // Deprecated: we don't use this field anymore after introducing pull based sink. + // Deprecated: we don't use this field anymore. MaxMemoryConsumption uint64 `toml:"max-memory-consumption" json:"max-memory-consumption"` - // the size of workerpool - // Deprecated: we don't use this field anymore after introducing pull based sink. + // Deprecated: we don't use this field anymore. NumWorkerPoolGoroutine int `toml:"num-workerpool-goroutine" json:"num-workerpool-goroutine"` - // number of concurrent heap sorts - // Deprecated: we don't use this field anymore after introducing pull based sink. + // Deprecated: we don't use this field anymore . NumConcurrentWorker int `toml:"num-concurrent-worker" json:"num-concurrent-worker"` - // maximum size for a heap - // Deprecated: we don't use this field anymore after introducing pull based sink. + // Deprecated: we don't use this field anymore. ChunkSizeLimit uint64 `toml:"chunk-size-limit" json:"chunk-size-limit"` } diff --git a/pkg/etcd/client.go b/pkg/etcd/client.go index 1e96c1546be..e63f775793f 100644 --- a/pkg/etcd/client.go +++ b/pkg/etcd/client.go @@ -15,6 +15,9 @@ package etcd import ( "context" + "crypto/tls" + "fmt" + "sync" "time" "github.com/benbjohnson/clock" @@ -23,11 +26,20 @@ import ( cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/errorutil" "github.com/pingcap/tiflow/pkg/retry" + "github.com/pingcap/tiflow/pkg/util" "github.com/prometheus/client_golang/prometheus" + "github.com/tikv/pd/pkg/errs" + "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" + "go.etcd.io/etcd/client/pkg/v3/logutil" clientV3 "go.etcd.io/etcd/client/v3" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" + "go.uber.org/zap/zapcore" + "google.golang.org/grpc" + "google.golang.org/grpc/backoff" "google.golang.org/grpc/codes" + "google.golang.org/grpc/keepalive" ) // etcd operation names @@ -313,3 +325,248 @@ func isRetryableError(rpcName string) retry.IsRetryable { return true } } + +// The following code is mainly copied from: +// https://github.com/tikv/pd/blob/master/pkg/utils/etcdutil/etcdutil.go +const ( + // defaultEtcdClientTimeout is the default timeout for etcd client. + defaultEtcdClientTimeout = 5 * time.Second + // defaultDialKeepAliveTime is the time after which client pings the server to see if transport is alive. + defaultDialKeepAliveTime = 10 * time.Second + // defaultDialKeepAliveTimeout is the time that the client waits for a response for the + // keep-alive probe. If the response is not received in this time, the connection is closed. + defaultDialKeepAliveTimeout = 3 * time.Second + // etcdServerOfflineTimeout is the timeout for an unhealthy etcd endpoint to be offline from healthy checker. + etcdServerOfflineTimeout = 30 * time.Minute + // etcdServerDisconnectedTimeout is the timeout for an unhealthy etcd endpoint to be disconnected from healthy checker. + etcdServerDisconnectedTimeout = 1 * time.Minute + // healthyPath is the path to check etcd health. + healthyPath = "health" +) + +func newClient(tlsConfig *tls.Config, grpcDialOption grpc.DialOption, endpoints ...string) (*clientv3.Client, error) { + if len(endpoints) == 0 { + return nil, errors.New("empty endpoints") + } + logConfig := logutil.DefaultZapLoggerConfig + logConfig.Level = zap.NewAtomicLevelAt(zapcore.ErrorLevel) + + lgc := zap.NewProductionConfig() + lgc.Encoding = log.ZapEncodingName + client, err := clientv3.New(clientv3.Config{ + Endpoints: endpoints, + TLS: tlsConfig, + LogConfig: &logConfig, + DialTimeout: defaultEtcdClientTimeout, + DialKeepAliveTime: defaultDialKeepAliveTime, + DialKeepAliveTimeout: defaultDialKeepAliveTimeout, + DialOptions: []grpc.DialOption{ + grpcDialOption, + grpc.WithBlock(), + grpc.WithConnectParams(grpc.ConnectParams{ + Backoff: backoff.Config{ + BaseDelay: time.Second, + Multiplier: 1.1, + Jitter: 0.1, + MaxDelay: 3 * time.Second, + }, + MinConnectTimeout: 3 * time.Second, + }), + grpc.WithKeepaliveParams(keepalive.ClientParameters{ + Time: 10 * time.Second, + Timeout: 20 * time.Second, + }), + }, + }) + if err != nil { + return nil, errors.Trace(err) + } + return client, nil +} + +// CreateRawEtcdClient creates etcd v3 client with detecting endpoints. +// It will check the health of endpoints periodically, and update endpoints if needed. +func CreateRawEtcdClient(tlsConfig *tls.Config, grpcDialOption grpc.DialOption, endpoints ...string) (*clientv3.Client, error) { + client, err := newClient(tlsConfig, grpcDialOption, endpoints...) + if err != nil { + return nil, err + } + + tickerInterval := defaultDialKeepAliveTime + + checker := &healthyChecker{ + tlsConfig: tlsConfig, + grpcDialOption: grpcDialOption, + } + eps := syncUrls(client) + checker.update(eps) + + // Create a goroutine to check the health of etcd endpoints periodically. + go func(client *clientv3.Client) { + ticker := time.NewTicker(tickerInterval) + defer ticker.Stop() + lastAvailable := time.Now() + for { + select { + case <-client.Ctx().Done(): + log.Info("etcd client is closed, exit health check goroutine") + checker.Range(func(key, value interface{}) bool { + client := value.(*healthyClient) + client.Close() + return true + }) + return + case <-ticker.C: + usedEps := client.Endpoints() + healthyEps := checker.patrol(client.Ctx()) + if len(healthyEps) == 0 { + // when all endpoints are unhealthy, try to reset endpoints to update connect + // rather than delete them to avoid there is no any endpoint in client. + // Note: reset endpoints will trigger subconn closed, and then trigger reconnect. + // otherwise, the subconn will be retrying in grpc layer and use exponential backoff, + // and it cannot recover as soon as possible. + if time.Since(lastAvailable) > etcdServerDisconnectedTimeout { + log.Info("no available endpoint, try to reset endpoints", zap.Strings("lastEndpoints", usedEps)) + client.SetEndpoints([]string{}...) + client.SetEndpoints(usedEps...) + } + } else { + if !util.AreStringSlicesEquivalent(healthyEps, usedEps) { + client.SetEndpoints(healthyEps...) + change := fmt.Sprintf("%d->%d", len(usedEps), len(healthyEps)) + etcdStateGauge.WithLabelValues("endpoints").Set(float64(len(healthyEps))) + log.Info("update endpoints", zap.String("numChange", change), + zap.Strings("lastEndpoints", usedEps), zap.Strings("endpoints", client.Endpoints())) + } + lastAvailable = time.Now() + } + } + } + }(client) + + // Notes: use another goroutine to update endpoints to avoid blocking health check in the first goroutine. + go func(client *clientv3.Client) { + ticker := time.NewTicker(tickerInterval) + defer ticker.Stop() + for { + select { + case <-client.Ctx().Done(): + log.Info("etcd client is closed, exit update endpoint goroutine") + return + case <-ticker.C: + eps := syncUrls(client) + checker.update(eps) + } + } + }(client) + + return client, nil +} + +type healthyClient struct { + *clientv3.Client + lastHealth time.Time +} + +type healthyChecker struct { + sync.Map // map[string]*healthyClient + tlsConfig *tls.Config + grpcDialOption grpc.DialOption +} + +func (checker *healthyChecker) patrol(ctx context.Context) []string { + // See https://github.com/etcd-io/etcd/blob/85b640cee793e25f3837c47200089d14a8392dc7/etcdctl/ctlv3/command/ep_command.go#L105-L145 + var wg sync.WaitGroup + count := 0 + checker.Range(func(key, value interface{}) bool { + count++ + return true + }) + hch := make(chan string, count) + healthyList := make([]string, 0, count) + checker.Range(func(key, value interface{}) bool { + wg.Add(1) + go func(key, value interface{}) { + defer wg.Done() + ep := key.(string) + client := value.(*healthyClient) + if IsHealthy(ctx, client.Client) { + hch <- ep + checker.Store(ep, &healthyClient{ + Client: client.Client, + lastHealth: time.Now(), + }) + return + } + }(key, value) + return true + }) + wg.Wait() + close(hch) + for h := range hch { + healthyList = append(healthyList, h) + } + return healthyList +} + +func (checker *healthyChecker) update(eps []string) { + for _, ep := range eps { + // check if client exists, if not, create one, if exists, check if it's offline or disconnected. + if client, ok := checker.Load(ep); ok { + lastHealthy := client.(*healthyClient).lastHealth + if time.Since(lastHealthy) > etcdServerOfflineTimeout { + log.Info("some etcd server maybe offline", zap.String("endpoint", ep)) + checker.Delete(ep) + } + if time.Since(lastHealthy) > etcdServerDisconnectedTimeout { + // try to reset client endpoint to trigger reconnect + client.(*healthyClient).Client.SetEndpoints([]string{}...) + client.(*healthyClient).Client.SetEndpoints(ep) + } + continue + } + checker.addClient(ep, time.Now()) + } +} + +func (checker *healthyChecker) addClient(ep string, lastHealth time.Time) { + client, err := newClient(checker.tlsConfig, checker.grpcDialOption, ep) + if err != nil { + log.Error("failed to create etcd healthy client", zap.Error(err)) + return + } + checker.Store(ep, &healthyClient{ + Client: client, + lastHealth: lastHealth, + }) +} + +func syncUrls(client *clientv3.Client) []string { + // See https://github.com/etcd-io/etcd/blob/85b640cee793e25f3837c47200089d14a8392dc7/clientv3/client.go#L170-L183 + ctx, cancel := context.WithTimeout(clientv3.WithRequireLeader(client.Ctx()), + etcdClientTimeoutDuration) + defer cancel() + mresp, err := client.MemberList(ctx) + if err != nil { + log.Error("failed to list members", errs.ZapError(err)) + return []string{} + } + var eps []string + for _, m := range mresp.Members { + if len(m.Name) != 0 && !m.IsLearner { + eps = append(eps, m.ClientURLs...) + } + } + return eps +} + +// IsHealthy checks if the etcd is healthy. +func IsHealthy(ctx context.Context, client *clientv3.Client) bool { + timeout := etcdClientTimeoutDuration + ctx, cancel := context.WithTimeout(clientv3.WithRequireLeader(ctx), timeout) + defer cancel() + _, err := client.Get(ctx, healthyPath) + // permission denied is OK since proposal goes through consensus to get it + // See: https://github.com/etcd-io/etcd/blob/85b640cee793e25f3837c47200089d14a8392dc7/etcdctl/ctlv3/command/ep_command.go#L124 + return err == nil || err == rpctypes.ErrPermissionDenied +} diff --git a/pkg/etcd/metrics.go b/pkg/etcd/metrics.go index 45690cadbc4..1f4e55e7f24 100644 --- a/pkg/etcd/metrics.go +++ b/pkg/etcd/metrics.go @@ -23,7 +23,16 @@ var etcdRequestCounter = prometheus.NewCounterVec( Help: "request counter of etcd operation", }, []string{"type"}) +var etcdStateGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "etcd", + Name: "etcd_client", + Help: "Etcd client states.", + }, []string{"type"}) + // InitMetrics registers the etcd request counter. func InitMetrics(registry *prometheus.Registry) { + prometheus.MustRegister(etcdStateGauge) registry.MustRegister(etcdRequestCounter) } diff --git a/pkg/migrate/migrate_test.go b/pkg/migrate/migrate_test.go index 929a8442cc8..96b0217f646 100644 --- a/pkg/migrate/migrate_test.go +++ b/pkg/migrate/migrate_test.go @@ -41,7 +41,7 @@ import ( const cycylicChangefeedInfo = `{"upstream-id":0,"sink-uri":"blackhole://","opts":{"a":"b"}, "create-time":"0001-01-01T00:00:00Z","start-ts":1,"target-ts":2,"admin-job-type":0,"sort-engine": -"memory","sort-dir":"/tmp/","config":{"case-sensitive":true,"enable-old-value":true, +"memory","sort-dir":"/tmp/","config":{"case-sensitive":true, "force-replicate":false,"check-gc-safe-point":true,"filter":{"rules":["*.*"], "ignore-txn-start-ts":null},"mounter":{"worker-num":16},"sink":{"dispatchers":null,"protocol":"", "column-selectors":null,"schema-registry":""},"cyclic-replication":{"enable":true,"replica-id":0, @@ -85,7 +85,6 @@ func TestMigration(t *testing.T) { } status2 := model.ChangeFeedStatus{CheckpointTs: 2} cfg := config.GetDefaultReplicaConfig() - cfg.EnableOldValue = false cfg.CheckGCSafePoint = false cfg.Sink = &config.SinkConfig{ DispatchRules: []*config.DispatchRule{ diff --git a/pkg/orchestrator/reactor_state_test.go b/pkg/orchestrator/reactor_state_test.go index a74ce331c74..46f4bff858a 100644 --- a/pkg/orchestrator/reactor_state_test.go +++ b/pkg/orchestrator/reactor_state_test.go @@ -56,7 +56,6 @@ func TestChangefeedStateUpdate(t *testing.T) { "sort-dir": "", "config": { "case-sensitive": true, - "enable-old-value": false, "force-replicate": false, "check-gc-safe-point": true, "filter": { diff --git a/pkg/sink/codec/avro/avro.go b/pkg/sink/codec/avro/avro.go index d6bc0ff00e4..80a863ba745 100644 --- a/pkg/sink/codec/avro/avro.go +++ b/pkg/sink/codec/avro/avro.go @@ -1039,6 +1039,9 @@ func (b *batchEncoderBuilder) Build() codec.RowEventEncoder { return NewAvroEncoder(b.namespace, b.schemaM, b.config) } +// CleanMetrics is a no-op for AvroEventBatchEncoder. +func (b *batchEncoderBuilder) CleanMetrics() {} + // NewAvroEncoder return a avro encoder. func NewAvroEncoder(namespace string, schemaM SchemaManager, config *common.Config) codec.RowEventEncoder { return &BatchEncoder{ diff --git a/pkg/sink/codec/avro/confluent_schema_registry.go b/pkg/sink/codec/avro/confluent_schema_registry.go index ced003b271a..9f06fe64587 100644 --- a/pkg/sink/codec/avro/confluent_schema_registry.go +++ b/pkg/sink/codec/avro/confluent_schema_registry.go @@ -368,7 +368,7 @@ func (m *confluentSchemaManager) ClearRegistry(ctx context.Context, schemaSubjec uri := m.registryURL + "/subjects/" + url.QueryEscape(schemaSubject) req, err := http.NewRequestWithContext(ctx, "DELETE", uri, nil) if err != nil { - log.Error("Could not construct request for clearRegistry", zap.String("uri", uri)) + log.Error("Could not construct request for clearRegistry", zap.Error(err)) return cerror.WrapError(cerror.ErrAvroSchemaAPIError, err) } req.Header.Add( diff --git a/pkg/sink/codec/builder/codec_test.go b/pkg/sink/codec/builder/codec_test.go index 58b96c21a13..785aaeab413 100644 --- a/pkg/sink/codec/builder/codec_test.go +++ b/pkg/sink/codec/builder/codec_test.go @@ -80,7 +80,9 @@ func TestJsonVsCraftVsPB(t *testing.T) { craftEncoder := craft.NewBatchEncoder(codecConfig) craftMessages := encodeRowCase(t, craftEncoder, cs) - jsonEncoder := open.NewBatchEncoder(codecConfig) + builder, err := open.NewBatchEncoderBuilder(context.Background(), codecConfig) + require.NoError(t, err) + jsonEncoder := builder.Build() jsonMessages := encodeRowCase(t, jsonEncoder, cs) protobuf1Messages := codecEncodeRowChangedPB1ToMessage(cs) @@ -237,7 +239,12 @@ func init() { panic(err) } - encoder = open.NewBatchEncoder(codecConfig) + builder, err := open.NewBatchEncoderBuilder(context.Background(), codecConfig) + if err != nil { + panic(err) + } + + encoder = builder.Build() if codecJSONEncodedRowChanges, err = codecEncodeRowCase(encoder, codecBenchmarkRowChanges); err != nil { panic(err) } @@ -260,7 +267,10 @@ func BenchmarkJsonEncoding(b *testing.B) { codecConfig := common.NewConfig(config.ProtocolCraft) codecConfig.MaxMessageBytes = 8192 codecConfig.MaxBatchSize = 64 - encoder := open.NewBatchEncoder(codecConfig) + + builder, err := open.NewBatchEncoderBuilder(context.Background(), codecConfig) + require.NoError(b, err) + encoder := builder.Build() for i := 0; i < b.N; i++ { _, _ = codecEncodeRowCase(encoder, codecBenchmarkRowChanges) } diff --git a/pkg/sink/codec/builder/encoder_builder.go b/pkg/sink/codec/builder/encoder_builder.go index 88943265f57..49e58543abc 100644 --- a/pkg/sink/codec/builder/encoder_builder.go +++ b/pkg/sink/codec/builder/encoder_builder.go @@ -30,12 +30,11 @@ import ( // NewRowEventEncoderBuilder returns an RowEventEncoderBuilder func NewRowEventEncoderBuilder( - ctx context.Context, - cfg *common.Config, + ctx context.Context, cfg *common.Config, ) (codec.RowEventEncoderBuilder, error) { switch cfg.Protocol { case config.ProtocolDefault, config.ProtocolOpen: - return open.NewBatchEncoderBuilder(cfg), nil + return open.NewBatchEncoderBuilder(ctx, cfg) case config.ProtocolCanal: return canal.NewBatchEncoderBuilder(cfg), nil case config.ProtocolAvro: @@ -43,7 +42,7 @@ func NewRowEventEncoderBuilder( case config.ProtocolMaxwell: return maxwell.NewBatchEncoderBuilder(cfg), nil case config.ProtocolCanalJSON: - return canal.NewJSONRowEventEncoderBuilder(cfg), nil + return canal.NewJSONRowEventEncoderBuilder(ctx, cfg) case config.ProtocolCraft: return craft.NewBatchEncoderBuilder(cfg), nil diff --git a/pkg/sink/codec/canal/canal_encoder.go b/pkg/sink/codec/canal/canal_encoder.go index 6e75e79d109..095c1410ceb 100644 --- a/pkg/sink/codec/canal/canal_encoder.go +++ b/pkg/sink/codec/canal/canal_encoder.go @@ -180,6 +180,9 @@ func (b *batchEncoderBuilder) Build() codec.RowEventEncoder { return newBatchEncoder(b.config) } +// CleanMetrics is a no-op for canalBatchEncoder. +func (b *batchEncoderBuilder) CleanMetrics() {} + // NewBatchEncoderBuilder creates a canal batchEncoderBuilder. func NewBatchEncoderBuilder(config *common.Config) codec.RowEventEncoderBuilder { return &batchEncoderBuilder{ diff --git a/pkg/sink/codec/canal/canal_json_decoder_test.go b/pkg/sink/codec/canal/canal_json_decoder_test.go index 9e274ba8da2..10d679278e0 100644 --- a/pkg/sink/codec/canal/canal_json_decoder_test.go +++ b/pkg/sink/codec/canal/canal_json_decoder_test.go @@ -32,10 +32,12 @@ func TestNewCanalJSONBatchDecoder4RowMessage(t *testing.T) { codecConfig := common.NewConfig(config.ProtocolCanalJSON) codecConfig.EnableTiDBExtension = encodeEnable codecConfig.Terminator = config.CRLF - encoder := newJSONRowEventEncoder(codecConfig) - require.NotNil(t, encoder) - err := encoder.AppendRowChangedEvent(ctx, "", testCaseInsert, nil) + builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + encoder := builder.Build() + + err = encoder.AppendRowChangedEvent(ctx, "", testCaseInsert, nil) require.NoError(t, err) messages := encoder.Build() @@ -95,7 +97,9 @@ func TestNewCanalJSONBatchDecoder4DDLMessage(t *testing.T) { codecConfig := common.NewConfig(config.ProtocolCanalJSON) codecConfig.EnableTiDBExtension = encodeEnable - encoder := newJSONRowEventEncoder(codecConfig) + builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + encoder := builder.Build() result, err := encoder.EncodeDDLEvent(testCaseDDL) require.NoError(t, err) diff --git a/pkg/sink/codec/canal/canal_json_row_event_encoder.go b/pkg/sink/codec/canal/canal_json_row_event_encoder.go index 91fa79e909a..5872484bba5 100644 --- a/pkg/sink/codec/canal/canal_json_row_event_encoder.go +++ b/pkg/sink/codec/canal/canal_json_row_event_encoder.go @@ -293,17 +293,21 @@ type JSONRowEventEncoder struct { builder *canalEntryBuilder messages []*common.Message + claimCheck *claimcheck.ClaimCheck + config *common.Config } // newJSONRowEventEncoder creates a new JSONRowEventEncoder -func newJSONRowEventEncoder(config *common.Config) codec.RowEventEncoder { - encoder := &JSONRowEventEncoder{ - builder: newCanalEntryBuilder(), - messages: make([]*common.Message, 0, 1), - config: config, +func newJSONRowEventEncoder( + config *common.Config, claimCheck *claimcheck.ClaimCheck, +) codec.RowEventEncoder { + return &JSONRowEventEncoder{ + builder: newCanalEntryBuilder(), + messages: make([]*common.Message, 0, 1), + config: config, + claimCheck: claimCheck, } - return encoder } func (c *JSONRowEventEncoder) newJSONMessageForDDL(e *model.DDLEvent) canalJSONMessageInterface { @@ -367,7 +371,7 @@ func (c *JSONRowEventEncoder) EncodeCheckpointEvent(ts uint64) (*common.Message, // AppendRowChangedEvent implements the interface EventJSONBatchEncoder func (c *JSONRowEventEncoder) AppendRowChangedEvent( - _ context.Context, + ctx context.Context, _ string, e *model.RowChangedEvent, callback func(), @@ -436,8 +440,15 @@ func (c *JSONRowEventEncoder) AppendRowChangedEvent( } if c.config.LargeMessageHandle.EnableClaimCheck() { - m.Event = e - m.ClaimCheckFileName = claimcheck.NewFileName() + claimCheckFileName := claimcheck.NewFileName() + if err := c.claimCheck.WriteMessage(ctx, m, claimCheckFileName); err != nil { + return errors.Trace(err) + } + + m, err = c.newClaimCheckLocationMessage(e, callback, claimCheckFileName) + if err != nil { + return errors.Trace(err) + } } } @@ -445,10 +456,11 @@ func (c *JSONRowEventEncoder) AppendRowChangedEvent( return nil } -// NewClaimCheckLocationMessage implements the ClaimCheckLocationEncoder interface -func (c *JSONRowEventEncoder) NewClaimCheckLocationMessage(origin *common.Message) (*common.Message, error) { - claimCheckLocation := claimcheck.FileNameWithPrefix(c.config.LargeMessageHandle.ClaimCheckStorageURI, origin.ClaimCheckFileName) - value, err := newJSONMessageForDML(c.builder, origin.Event, c.config, true, claimCheckLocation) +func (c *JSONRowEventEncoder) newClaimCheckLocationMessage( + event *model.RowChangedEvent, callback func(), fileName string, +) (*common.Message, error) { + claimCheckLocation := c.claimCheck.FileNameWithPrefix(fileName) + value, err := newJSONMessageForDML(c.builder, event, c.config, true, claimCheckLocation) if err != nil { return nil, cerror.WrapError(cerror.ErrCanalEncodeFailed, err) } @@ -461,7 +473,7 @@ func (c *JSONRowEventEncoder) NewClaimCheckLocationMessage(origin *common.Messag } result := common.NewMsg(config.ProtocolCanalJSON, nil, value, 0, model.MessageTypeRow, nil, nil) - result.Callback = origin.Callback + result.Callback = callback result.IncRowsCount() length := result.Length() @@ -469,7 +481,7 @@ func (c *JSONRowEventEncoder) NewClaimCheckLocationMessage(origin *common.Messag log.Warn("Single message is too large for canal-json, when create the claim check location message", zap.Int("maxMessageBytes", c.config.MaxMessageBytes), zap.Int("length", length), - zap.Any("table", origin.Event.Table)) + zap.Any("table", event.Table)) return nil, cerror.ErrMessageTooLarge.GenWithStackByArgs(length) } return result, nil @@ -504,16 +516,31 @@ func (c *JSONRowEventEncoder) EncodeDDLEvent(e *model.DDLEvent) (*common.Message type jsonRowEventEncoderBuilder struct { config *common.Config + + claimCheck *claimcheck.ClaimCheck } // NewJSONRowEventEncoderBuilder creates a canal-json batchEncoderBuilder. -func NewJSONRowEventEncoderBuilder(config *common.Config) codec.RowEventEncoderBuilder { - return &jsonRowEventEncoderBuilder{config: config} +func NewJSONRowEventEncoderBuilder(ctx context.Context, config *common.Config) (codec.RowEventEncoderBuilder, error) { + var ( + claimCheck *claimcheck.ClaimCheck + err error + ) + if config.LargeMessageHandle.EnableClaimCheck() { + claimCheck, err = claimcheck.New(ctx, config.LargeMessageHandle.ClaimCheckStorageURI, config.ChangefeedID) + if err != nil { + return nil, errors.Trace(err) + } + } + return &jsonRowEventEncoderBuilder{ + config: config, + claimCheck: claimCheck, + }, nil } // Build a `jsonRowEventEncoderBuilder` func (b *jsonRowEventEncoderBuilder) Build() codec.RowEventEncoder { - return newJSONRowEventEncoder(b.config) + return newJSONRowEventEncoder(b.config, b.claimCheck) } func shouldIgnoreColumn(col *model.Column, @@ -532,3 +559,9 @@ func shouldIgnoreColumn(col *model.Column, } return false } + +func (b *jsonRowEventEncoderBuilder) CleanMetrics() { + if b.claimCheck != nil { + b.claimCheck.CleanMetrics() + } +} diff --git a/pkg/sink/codec/canal/canal_json_row_event_encoder_test.go b/pkg/sink/codec/canal/canal_json_row_event_encoder_test.go index d5b7b92cf01..fd7ba7ac81d 100644 --- a/pkg/sink/codec/canal/canal_json_row_event_encoder_test.go +++ b/pkg/sink/codec/canal/canal_json_row_event_encoder_test.go @@ -17,7 +17,6 @@ import ( "context" "database/sql" "encoding/json" - "path/filepath" "testing" "github.com/pingcap/tidb/parser/mysql" @@ -25,9 +24,7 @@ import ( "github.com/pingcap/tiflow/pkg/compression" "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/sink/codec" "github.com/pingcap/tiflow/pkg/sink/codec/common" - "github.com/pingcap/tiflow/pkg/sink/kafka/claimcheck" "github.com/stretchr/testify/require" "golang.org/x/text/encoding/charmap" ) @@ -45,9 +42,12 @@ func TestBuildCanalJSONRowEventEncoder(t *testing.T) { func TestNewCanalJSONMessage4DML(t *testing.T) { t.Parallel() + ctx := context.Background() codecConfig := common.NewConfig(config.ProtocolCanalJSON) - e := newJSONRowEventEncoder(codecConfig) - require.NotNil(t, e) + builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + + e := builder.Build() encoder, ok := e.(*JSONRowEventEncoder) require.True(t, ok) @@ -95,7 +95,7 @@ func TestNewCanalJSONMessage4DML(t *testing.T) { if bytes, ok := item.column.Value.([]byte); ok { expectedValue, err := charmap.ISO8859_1.NewDecoder().Bytes(bytes) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, string(expectedValue), obtainedValue) continue } @@ -162,8 +162,10 @@ func TestNewCanalJSONMessage4DML(t *testing.T) { codecConfig = common.NewConfig(config.ProtocolCanalJSON) codecConfig.EnableTiDBExtension = true codecConfig.OnlyOutputUpdatedColumns = true - e = newJSONRowEventEncoder(codecConfig) - require.NotNil(t, e) + + builder, err = NewJSONRowEventEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + e = builder.Build() encoder, ok = e.(*JSONRowEventEncoder) require.True(t, ok) @@ -198,11 +200,13 @@ func TestCanalJSONCompressionE2E(t *testing.T) { codecConfig.EnableTiDBExtension = true codecConfig.LargeMessageHandle.LargeMessageHandleCompression = compression.LZ4 - encoder := newJSONRowEventEncoder(codecConfig) + ctx := context.Background() + builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + encoder := builder.Build() // encode normal row changed event - ctx := context.Background() - err := encoder.AppendRowChangedEvent(ctx, "", testCaseInsert, func() {}) + err = encoder.AppendRowChangedEvent(ctx, "", testCaseInsert, func() {}) require.NoError(t, err) message := encoder.Build()[0] @@ -271,37 +275,17 @@ func TestCanalJSONClaimCheckE2E(t *testing.T) { codecConfig.LargeMessageHandle.LargeMessageHandleCompression = compression.Snappy codecConfig.LargeMessageHandle.ClaimCheckStorageURI = "file:///tmp/canal-json-claim-check" codecConfig.MaxMessageBytes = 500 - - encoder := newJSONRowEventEncoder(codecConfig) - ctx := context.Background() - err := encoder.AppendRowChangedEvent(ctx, "", testCaseInsert, func() {}) - require.NoError(t, err) - - // this is a large message, should be delivered to the external storage. - largeMessage := encoder.Build()[0] - require.NotEmpty(t, largeMessage.ClaimCheckFileName) - // the message delivered to the kafka - claimCheckLocationMessage, err := encoder.(codec.ClaimCheckLocationEncoder).NewClaimCheckLocationMessage(largeMessage) + builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig) require.NoError(t, err) - require.Empty(t, claimCheckLocationMessage.ClaimCheckFileName) + encoder := builder.Build() - value, err := common.Decompress(compression.Snappy, claimCheckLocationMessage.Value) + err = encoder.AppendRowChangedEvent(ctx, "", testCaseInsert, func() {}) require.NoError(t, err) - var decoded canalJSONMessageWithTiDBExtension - err = json.Unmarshal(value, &decoded) - require.NoError(t, err) - _, claimCheckFilename := filepath.Split(decoded.Extensions.ClaimCheckLocation) - require.Equal(t, largeMessage.ClaimCheckFileName, claimCheckFilename) - - changefeedID := model.DefaultChangeFeedID("claim-check-test") - claimCheckStorage, err := claimcheck.New(ctx, codecConfig.LargeMessageHandle.ClaimCheckStorageURI, changefeedID) - require.NoError(t, err) - defer claimCheckStorage.Close() - err = claimCheckStorage.WriteMessage(ctx, largeMessage) - require.NoError(t, err) + // this is a large message, should be delivered to the external storage. + claimCheckLocationMessage := encoder.Build()[0] decoder, err := NewBatchDecoder(ctx, codecConfig, nil) require.NoError(t, err) @@ -343,9 +327,14 @@ func TestNewCanalJSONMessageHandleKeyOnly4LargeMessage(t *testing.T) { codecConfig.LargeMessageHandle.LargeMessageHandleOption = config.LargeMessageHandleOptionHandleKeyOnly codecConfig.LargeMessageHandle.LargeMessageHandleCompression = compression.LZ4 codecConfig.MaxMessageBytes = 500 - encoder := newJSONRowEventEncoder(codecConfig) - err := encoder.AppendRowChangedEvent(context.Background(), "", testCaseInsert, func() {}) + ctx := context.Background() + + builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + encoder := builder.Build() + + err = encoder.AppendRowChangedEvent(context.Background(), "", testCaseInsert, func() {}) require.NoError(t, err) message := encoder.Build()[0] @@ -380,8 +369,12 @@ func TestNewCanalJSONMessageHandleKeyOnly4LargeMessage(t *testing.T) { func TestNewCanalJSONMessageFromDDL(t *testing.T) { t.Parallel() - encoder, ok := newJSONRowEventEncoder(&common.Config{}).(*JSONRowEventEncoder) - require.True(t, ok) + codecConfig := common.NewConfig(config.ProtocolCanalJSON) + ctx := context.Background() + + builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + encoder := builder.Build().(*JSONRowEventEncoder) message := encoder.newJSONMessageForDDL(testCaseDDL) require.NotNil(t, message) @@ -395,11 +388,11 @@ func TestNewCanalJSONMessageFromDDL(t *testing.T) { require.Equal(t, testCaseDDL.Query, msg.Query) require.Equal(t, "CREATE", msg.EventType) - encoder, ok = newJSONRowEventEncoder(&common.Config{ - EnableTiDBExtension: true, - }).(*JSONRowEventEncoder) - require.True(t, ok) + codecConfig.EnableTiDBExtension = true + builder, err = NewJSONRowEventEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + encoder = builder.Build().(*JSONRowEventEncoder) message = encoder.newJSONMessageForDDL(testCaseDDL) require.NotNil(t, message) @@ -413,8 +406,11 @@ func TestNewCanalJSONMessageFromDDL(t *testing.T) { func TestBatching(t *testing.T) { t.Parallel() + ctx := context.Background() codecConfig := common.NewConfig(config.ProtocolCanalJSON) - encoder := newJSONRowEventEncoder(codecConfig) + builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + encoder := builder.Build() require.NotNil(t, encoder) updateCase := *testCaseUpdate @@ -445,13 +441,17 @@ func TestBatching(t *testing.T) { func TestEncodeCheckpointEvent(t *testing.T) { t.Parallel() + + ctx := context.Background() var watermark uint64 = 2333 for _, enable := range []bool{false, true} { codecConfig := common.NewConfig(config.ProtocolCanalJSON) codecConfig.EnableTiDBExtension = enable - encoder := newJSONRowEventEncoder(codecConfig) - require.NotNil(t, encoder) + builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + + encoder := builder.Build() msg, err := encoder.EncodeCheckpointEvent(watermark) require.NoError(t, err) @@ -496,9 +496,12 @@ func TestCheckpointEventValueMarshal(t *testing.T) { codecConfig := common.NewConfig(config.ProtocolCanalJSON) codecConfig.EnableTiDBExtension = true - encoder := newJSONRowEventEncoder(codecConfig) - require.NotNil(t, encoder) + ctx := context.Background() + + builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + encoder := builder.Build() var watermark uint64 = 1024 msg, err := encoder.EncodeCheckpointEvent(watermark) require.NoError(t, err) @@ -584,11 +587,13 @@ func TestDDLEventWithExtensionValueMarshal(t *testing.T) { func TestCanalJSONAppendRowChangedEventWithCallback(t *testing.T) { codecConfig := common.NewConfig(config.ProtocolCanalJSON) codecConfig.EnableTiDBExtension = true - encoder := newJSONRowEventEncoder(codecConfig) - require.NotNil(t, encoder) + ctx := context.Background() - count := 0 + builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + encoder := builder.Build() + count := 0 row := &model.RowChangedEvent{ CommitTs: 1, Table: &model.TableName{Schema: "a", Table: "b"}, @@ -677,14 +682,22 @@ func TestMaxMessageBytes(t *testing.T) { // the test message length is smaller than max-message-bytes maxMessageBytes := 300 - cfg := common.NewConfig(config.ProtocolCanalJSON).WithMaxMessageBytes(maxMessageBytes) - encoder := NewJSONRowEventEncoderBuilder(cfg).Build() - err := encoder.AppendRowChangedEvent(ctx, topic, testEvent, nil) + codecConfig := common.NewConfig(config.ProtocolCanalJSON).WithMaxMessageBytes(maxMessageBytes) + + builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + encoder := builder.Build() + + err = encoder.AppendRowChangedEvent(ctx, topic, testEvent, nil) require.NoError(t, err) // the test message length is larger than max-message-bytes - cfg = cfg.WithMaxMessageBytes(100) - encoder = NewJSONRowEventEncoderBuilder(cfg).Build() + codecConfig = codecConfig.WithMaxMessageBytes(100) + + builder, err = NewJSONRowEventEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + + encoder = builder.Build() err = encoder.AppendRowChangedEvent(ctx, topic, testEvent, nil) require.Error(t, err, cerror.ErrMessageTooLarge) } diff --git a/pkg/sink/codec/common/config.go b/pkg/sink/codec/common/config.go index f7f2743e059..8e7ba8f241a 100644 --- a/pkg/sink/codec/common/config.go +++ b/pkg/sink/codec/common/config.go @@ -100,8 +100,6 @@ const ( codecOPTAvroBigintUnsignedHandlingMode = "avro-bigint-unsigned-handling-mode" codecOPTAvroSchemaRegistry = "schema-registry" coderOPTAvroGlueSchemaRegistry = "glue-schema-registry" - - codecOPTOnlyOutputUpdatedColumns = "only-output-updated-columns" ) const ( @@ -190,7 +188,7 @@ func (c *Config) Apply(sinkURI *url.URL, replicaConfig *config.ReplicaConfig) er c.IncludeCommitTs = replicaConfig.Sink.CSVConfig.IncludeCommitTs c.BinaryEncodingMethod = replicaConfig.Sink.CSVConfig.BinaryEncodingMethod } - if replicaConfig.Sink.KafkaConfig != nil { + if replicaConfig.Sink.KafkaConfig != nil && replicaConfig.Sink.KafkaConfig.LargeMessageHandle != nil { c.LargeMessageHandle = replicaConfig.Sink.KafkaConfig.LargeMessageHandle } if !c.LargeMessageHandle.Disabled() && replicaConfig.ForceReplicate { @@ -202,12 +200,6 @@ func (c *Config) Apply(sinkURI *url.URL, replicaConfig *config.ReplicaConfig) er if urlParameter.OnlyOutputUpdatedColumns != nil { c.OnlyOutputUpdatedColumns = *urlParameter.OnlyOutputUpdatedColumns } - if c.OnlyOutputUpdatedColumns && !replicaConfig.EnableOldValue { - return cerror.ErrCodecInvalidConfig.GenWithStack( - `old value must be enabled when configuration "%s" is true.`, - codecOPTOnlyOutputUpdatedColumns, - ) - } if replicaConfig.Integrity != nil { c.EnableRowChecksum = replicaConfig.Integrity.Enabled() @@ -332,7 +324,7 @@ func (c *Config) Validate() error { } if c.LargeMessageHandle != nil { - err := c.LargeMessageHandle.Validate(c.Protocol, c.EnableTiDBExtension) + err := c.LargeMessageHandle.AdjustAndValidate(c.Protocol, c.EnableTiDBExtension) if err != nil { return err } diff --git a/pkg/sink/codec/common/message.go b/pkg/sink/codec/common/message.go index ab439ba9971..032416a7402 100644 --- a/pkg/sink/codec/common/message.go +++ b/pkg/sink/codec/common/message.go @@ -42,12 +42,6 @@ type Message struct { rowsCount int // rows in one Message Callback func() // Callback function will be called when the message is sent to the sink. - // ClaimCheckFileName is set if the message should be sent to the claim check storage. - // it's only the file name, since the claim check storage writer know the path. - ClaimCheckFileName string - - Event *model.RowChangedEvent - // PartitionKey for pulsar, route messages to one or different partitions PartitionKey *string } @@ -95,6 +89,12 @@ func (m *Message) GetTable() string { return *m.Table } +// SetPartitionKey sets the PartitionKey for a message +// PartitionKey is used for pulsar producer, route messages to one or different partitions +func (m *Message) SetPartitionKey(key string) { + m.PartitionKey = &key +} + // GetPartitionKey returns the GetPartitionKey func (m *Message) GetPartitionKey() string { if m.PartitionKey == nil { diff --git a/pkg/sink/codec/craft/craft_encoder.go b/pkg/sink/codec/craft/craft_encoder.go index 5de08131741..0b9ace17311 100644 --- a/pkg/sink/codec/craft/craft_encoder.go +++ b/pkg/sink/codec/craft/craft_encoder.go @@ -112,6 +112,9 @@ func (b *batchEncoderBuilder) Build() codec.RowEventEncoder { return NewBatchEncoder(b.config) } +// CleanMetrics do nothing +func (b *batchEncoderBuilder) CleanMetrics() {} + // NewBatchEncoderBuilder creates a craft batchEncoderBuilder. func NewBatchEncoderBuilder(config *common.Config) codec.RowEventEncoderBuilder { return &batchEncoderBuilder{config: config} diff --git a/pkg/sink/codec/encoder.go b/pkg/sink/codec/encoder.go index a18eec3cd69..3ddc345559e 100644 --- a/pkg/sink/codec/encoder.go +++ b/pkg/sink/codec/encoder.go @@ -50,14 +50,10 @@ type RowEventEncoder interface { MessageBuilder } -// ClaimCheckLocationEncoder is an abstraction for claim check encoder. -type ClaimCheckLocationEncoder interface { - NewClaimCheckLocationMessage(origin *common.Message) (*common.Message, error) -} - // RowEventEncoderBuilder builds row encoder with context. type RowEventEncoderBuilder interface { Build() RowEventEncoder + CleanMetrics() } // TxnEventEncoder is an abstraction for txn events encoder. diff --git a/pkg/sink/codec/encoder_group.go b/pkg/sink/codec/encoder_group.go index 2ce9a13b0a4..82f818a7521 100644 --- a/pkg/sink/codec/encoder_group.go +++ b/pkg/sink/codec/encoder_group.go @@ -23,15 +23,15 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink/dmlsink" + "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/sink/codec/common" "go.uber.org/zap" "golang.org/x/sync/errgroup" ) const ( - defaultEncoderGroupSize = 16 - defaultInputChanSize = 256 - defaultMetricInterval = 15 * time.Second + defaultInputChanSize = 128 + defaultMetricInterval = 15 * time.Second ) // EncoderGroup manages a group of encoders @@ -41,7 +41,7 @@ type EncoderGroup interface { // AddEvents add events into the group, handled by one of the encoders // all input events should belong to the same topic and partition, this should be guaranteed by the caller AddEvents(ctx context.Context, topic string, partition int32, - events ...*dmlsink.RowChangeCallbackableEvent) error + partitionKey string, events ...*dmlsink.RowChangeCallbackableEvent) error // Output returns a channel produce futures Output() <-chan *future } @@ -62,7 +62,7 @@ func NewEncoderGroup(builder RowEventEncoderBuilder, count int, changefeedID model.ChangeFeedID, ) *encoderGroup { if count <= 0 { - count = defaultEncoderGroupSize + count = config.DefaultEncoderGroupConcurrency } inputCh := make([]chan *future, count) @@ -83,7 +83,7 @@ func NewEncoderGroup(builder RowEventEncoderBuilder, func (g *encoderGroup) Run(ctx context.Context) error { defer func() { - encoderGroupInputChanSizeGauge.DeleteLabelValues(g.changefeedID.Namespace, g.changefeedID.ID) + g.cleanMetrics() log.Info("encoder group exited", zap.String("namespace", g.changefeedID.Namespace), zap.String("changefeed", g.changefeedID.ID)) @@ -128,9 +128,10 @@ func (g *encoderGroup) AddEvents( ctx context.Context, topic string, partition int32, + partitionKey string, events ...*dmlsink.RowChangeCallbackableEvent, ) error { - future := newFuture(topic, partition, events...) + future := newFuture(topic, partition, partitionKey, events...) index := atomic.AddUint64(&g.index, 1) % uint64(g.count) select { case <-ctx.Done(): @@ -151,22 +152,30 @@ func (g *encoderGroup) Output() <-chan *future { return g.outputCh } +func (g *encoderGroup) cleanMetrics() { + encoderGroupInputChanSizeGauge.DeleteLabelValues(g.changefeedID.Namespace, g.changefeedID.ID) + g.builder.CleanMetrics() + common.CleanMetrics(g.changefeedID) +} + type future struct { - Topic string - Partition int32 - events []*dmlsink.RowChangeCallbackableEvent - Messages []*common.Message + Topic string + Partition int32 + PartitionKey string + events []*dmlsink.RowChangeCallbackableEvent + Messages []*common.Message done chan struct{} } -func newFuture(topic string, partition int32, +func newFuture(topic string, partition int32, partitionKey string, events ...*dmlsink.RowChangeCallbackableEvent, ) *future { return &future{ - Topic: topic, - Partition: partition, - events: events, + Topic: topic, + Partition: partition, + PartitionKey: partitionKey, + events: events, done: make(chan struct{}), } diff --git a/pkg/sink/codec/maxwell/maxwell_encoder.go b/pkg/sink/codec/maxwell/maxwell_encoder.go index 1f175ac132e..a60a4bb0e64 100644 --- a/pkg/sink/codec/maxwell/maxwell_encoder.go +++ b/pkg/sink/codec/maxwell/maxwell_encoder.go @@ -137,3 +137,6 @@ func NewBatchEncoderBuilder(config *common.Config) codec.RowEventEncoderBuilder func (b *batchEncoderBuilder) Build() codec.RowEventEncoder { return newBatchEncoder(b.config) } + +// CleanMetrics do nothing +func (b *batchEncoderBuilder) CleanMetrics() {} diff --git a/pkg/sink/codec/open/open_protocol_encoder.go b/pkg/sink/codec/open/open_protocol_encoder.go index 0ec24dcb971..6f1ecfb3100 100644 --- a/pkg/sink/codec/open/open_protocol_encoder.go +++ b/pkg/sink/codec/open/open_protocol_encoder.go @@ -35,6 +35,8 @@ type BatchEncoder struct { callbackBuff []func() curBatchSize int + claimCheck *claimcheck.ClaimCheck + config *common.Config } @@ -77,7 +79,7 @@ func (d *BatchEncoder) buildMessageOnlyHandleKeyColumns(e *model.RowChangedEvent // AppendRowChangedEvent implements the RowEventEncoder interface func (d *BatchEncoder) AppendRowChangedEvent( - _ context.Context, + ctx context.Context, _ string, e *model.RowChangedEvent, callback func(), @@ -119,7 +121,9 @@ func (d *BatchEncoder) AppendRowChangedEvent( if d.config.LargeMessageHandle.EnableClaimCheck() { // build previous batched messages d.tryBuildCallback() - d.appendSingleLargeMessage4ClaimCheck(key, value, e, callback) + if err := d.appendSingleLargeMessage4ClaimCheck(ctx, key, value, e, callback); err != nil { + return errors.Trace(err) + } return nil } @@ -257,14 +261,16 @@ func (d *BatchEncoder) tryBuildCallback() { } // NewClaimCheckLocationMessage implement the ClaimCheckLocationEncoder interface. -func (d *BatchEncoder) NewClaimCheckLocationMessage(origin *common.Message) (*common.Message, error) { - keyMsg, valueMsg, err := rowChangeToMsg(origin.Event, d.config, true) +func (d *BatchEncoder) newClaimCheckLocationMessage( + event *model.RowChangedEvent, callback func(), fileName string, +) (*common.Message, error) { + keyMsg, valueMsg, err := rowChangeToMsg(event, d.config, true) if err != nil { return nil, errors.Trace(err) } keyMsg.OnlyHandleKey = false - claimCheckLocation := claimcheck.FileNameWithPrefix(d.config.LargeMessageHandle.ClaimCheckStorageURI, origin.ClaimCheckFileName) + claimCheckLocation := d.claimCheck.FileNameWithPrefix(fileName) keyMsg.ClaimCheckLocation = claimCheckLocation key, err := keyMsg.Encode() if err != nil { @@ -296,29 +302,37 @@ func (d *BatchEncoder) NewClaimCheckLocationMessage(origin *common.Message) (*co } message := newMessage(key, value) - message.Ts = origin.Ts - message.Schema = origin.Schema - message.Table = origin.Table + message.Ts = event.CommitTs + message.Schema = &event.Table.Schema + message.Table = &event.Table.Table message.IncRowsCount() - if origin.Callback != nil { - message.Callback = origin.Callback + if callback != nil { + message.Callback = callback } return message, nil } -func (d *BatchEncoder) appendSingleLargeMessage4ClaimCheck(key, value []byte, e *model.RowChangedEvent, callback func()) { +func (d *BatchEncoder) appendSingleLargeMessage4ClaimCheck( + ctx context.Context, key, value []byte, e *model.RowChangedEvent, callback func(), +) error { message := newMessage(key, value) message.Ts = e.CommitTs message.Schema = &e.Table.Schema message.Table = &e.Table.Table - // ClaimCheckFileName must be set to indicate this message should be sent to the external storage. - message.ClaimCheckFileName = claimcheck.NewFileName() - message.Event = e message.IncRowsCount() - if callback != nil { - message.Callback = callback + + claimCheckFileName := claimcheck.NewFileName() + if err := d.claimCheck.WriteMessage(ctx, message, claimCheckFileName); err != nil { + return errors.Trace(err) + } + + message, err := d.newClaimCheckLocationMessage(e, callback, claimCheckFileName) + if err != nil { + return errors.Trace(err) } d.messageBuf = append(d.messageBuf, message) + + return nil } func newMessage(key, value []byte) *common.Message { @@ -342,22 +356,45 @@ func newMessage(key, value []byte) *common.Message { } type batchEncoderBuilder struct { - config *common.Config + claimCheck *claimcheck.ClaimCheck + config *common.Config } // Build a BatchEncoder func (b *batchEncoderBuilder) Build() codec.RowEventEncoder { - return NewBatchEncoder(b.config) + return NewBatchEncoder(b.config, b.claimCheck) +} + +func (b *batchEncoderBuilder) CleanMetrics() { + if b.claimCheck != nil { + b.claimCheck.CleanMetrics() + } } // NewBatchEncoderBuilder creates an open-protocol batchEncoderBuilder. -func NewBatchEncoderBuilder(config *common.Config) codec.RowEventEncoderBuilder { - return &batchEncoderBuilder{config: config} +func NewBatchEncoderBuilder( + ctx context.Context, config *common.Config, +) (codec.RowEventEncoderBuilder, error) { + var ( + claimCheck *claimcheck.ClaimCheck + err error + ) + if config.LargeMessageHandle.EnableClaimCheck() { + claimCheck, err = claimcheck.New(ctx, config.LargeMessageHandle.ClaimCheckStorageURI, config.ChangefeedID) + if err != nil { + return nil, errors.Trace(err) + } + } + return &batchEncoderBuilder{ + config: config, + claimCheck: claimCheck, + }, nil } // NewBatchEncoder creates a new BatchEncoder. -func NewBatchEncoder(config *common.Config) codec.RowEventEncoder { +func NewBatchEncoder(config *common.Config, claimCheck *claimcheck.ClaimCheck) codec.RowEventEncoder { return &BatchEncoder{ - config: config, + config: config, + claimCheck: claimCheck, } } diff --git a/pkg/sink/codec/open/open_protocol_encoder_test.go b/pkg/sink/codec/open/open_protocol_encoder_test.go index 6eea9242f03..e86c399501f 100644 --- a/pkg/sink/codec/open/open_protocol_encoder_test.go +++ b/pkg/sink/codec/open/open_protocol_encoder_test.go @@ -16,7 +16,6 @@ package open import ( "context" "database/sql" - "path/filepath" "testing" timodel "github.com/pingcap/tidb/parser/model" @@ -28,7 +27,6 @@ import ( "github.com/pingcap/tiflow/pkg/sink/codec" "github.com/pingcap/tiflow/pkg/sink/codec/common" "github.com/pingcap/tiflow/pkg/sink/codec/internal" - "github.com/pingcap/tiflow/pkg/sink/kafka/claimcheck" "github.com/stretchr/testify/require" ) @@ -162,19 +160,25 @@ func TestMaxMessageBytes(t *testing.T) { // just can hold it. a := 173 codecConfig := common.NewConfig(config.ProtocolOpen).WithMaxMessageBytes(a) - encoder := NewBatchEncoderBuilder(codecConfig).Build() - err := encoder.AppendRowChangedEvent(ctx, topic, testEvent, nil) + builder, err := NewBatchEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + encoder := builder.Build() + err = encoder.AppendRowChangedEvent(ctx, topic, testEvent, nil) require.NoError(t, err) // cannot hold a single message codecConfig = codecConfig.WithMaxMessageBytes(a - 1) - encoder = NewBatchEncoderBuilder(codecConfig).Build() + builder, err = NewBatchEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + encoder = builder.Build() err = encoder.AppendRowChangedEvent(ctx, topic, testEvent, nil) require.ErrorIs(t, err, cerror.ErrMessageTooLarge) // make sure each batch's `Length` not greater than `max-message-bytes` codecConfig = codecConfig.WithMaxMessageBytes(256) - encoder = NewBatchEncoderBuilder(codecConfig).Build() + builder, err = NewBatchEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + encoder = builder.Build() for i := 0; i < 10000; i++ { err := encoder.AppendRowChangedEvent(ctx, topic, testEvent, nil) require.NoError(t, err) @@ -188,12 +192,16 @@ func TestMaxMessageBytes(t *testing.T) { func TestMaxBatchSize(t *testing.T) { t.Parallel() + + ctx := context.Background() codecConfig := common.NewConfig(config.ProtocolOpen).WithMaxMessageBytes(1048576) codecConfig.MaxBatchSize = 64 - encoder := NewBatchEncoderBuilder(codecConfig).Build() + builder, err := NewBatchEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + encoder := builder.Build() for i := 0; i < 10000; i++ { - err := encoder.AppendRowChangedEvent(context.Background(), "", testEvent, nil) + err := encoder.AppendRowChangedEvent(ctx, "", testEvent, nil) require.NoError(t, err) } @@ -297,7 +305,9 @@ func TestOpenProtocolBatchCodec(t *testing.T) { codecConfig := common.NewConfig(config.ProtocolOpen).WithMaxMessageBytes(8192) codecConfig.MaxBatchSize = 64 tester := internal.NewDefaultBatchTester() - tester.TestBatchCodec(t, NewBatchEncoderBuilder(codecConfig), + builder, err := NewBatchEncoderBuilder(context.Background(), codecConfig) + require.NoError(t, err) + tester.TestBatchCodec(t, builder, func(key []byte, value []byte) (codec.RowEventDecoder, error) { decoder, err := NewBatchDecoder(context.Background(), codecConfig, nil) require.NoError(t, err) @@ -313,10 +323,11 @@ func TestEncodeDecodeE2E(t *testing.T) { topic := "test" codecConfig := common.NewConfig(config.ProtocolOpen) + builder, err := NewBatchEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + encoder := builder.Build() - encoder := NewBatchEncoder(codecConfig) - - err := encoder.AppendRowChangedEvent(ctx, topic, testEvent, func() {}) + err = encoder.AppendRowChangedEvent(ctx, topic, testEvent, func() {}) require.NoError(t, err) message := encoder.Build()[0] @@ -353,7 +364,9 @@ func TestE2EDDLCompression(t *testing.T) { codecConfig := common.NewConfig(config.ProtocolOpen) codecConfig.LargeMessageHandle.LargeMessageHandleCompression = compression.Snappy - encoder := NewBatchEncoder(codecConfig) + builder, err := NewBatchEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + encoder := builder.Build() // encode DDL event message, err := encoder.EncodeDDLEvent(testCaseDDL) @@ -405,11 +418,13 @@ func TestE2EHandleKeyOnlyEvent(t *testing.T) { codecConfig.MaxMessageBytes = 251 - encoder := NewBatchEncoder(codecConfig) - ctx := context.Background() + builder, err := NewBatchEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + encoder := builder.Build() + topic := "test" - err := encoder.AppendRowChangedEvent(ctx, topic, largeTestEvent, nil) + err = encoder.AppendRowChangedEvent(ctx, topic, largeTestEvent, nil) require.NoError(t, err) message := encoder.Build()[0] @@ -452,12 +467,15 @@ func TestE2EClaimCheckMessage(t *testing.T) { codecConfig.LargeMessageHandle.LargeMessageHandleOption = config.LargeMessageHandleOptionClaimCheck codecConfig.LargeMessageHandle.LargeMessageHandleCompression = compression.LZ4 codecConfig.LargeMessageHandle.ClaimCheckStorageURI = "file:///tmp/claim-check" - encoder := NewBatchEncoderBuilder(codecConfig).Build() - err := encoder.AppendRowChangedEvent(ctx, topic, testEvent, func() {}) + builder, err := NewBatchEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + encoder := builder.Build() + + err = encoder.AppendRowChangedEvent(ctx, topic, testEvent, func() {}) require.NoError(t, err) - // cannot hold this message, encode it as claim check message by force. + // cannot hold this message, it's encoded as the claim check location message. err = encoder.AppendRowChangedEvent(ctx, topic, largeTestEvent, func() {}) require.NoError(t, err) @@ -467,26 +485,7 @@ func TestE2EClaimCheckMessage(t *testing.T) { messages := encoder.Build() require.Len(t, messages, 3) - require.Empty(t, messages[0].ClaimCheckFileName) - require.NotEmptyf(t, messages[1].ClaimCheckFileName, "claim check file name should not be empty") - require.Empty(t, messages[2].ClaimCheckFileName) - - // write the large message to the external storage, the local filesystem in this case. - largeMessage := messages[1] - - changefeedID := model.DefaultChangeFeedID("claim-check-test") - claimCheckStorage, err := claimcheck.New(ctx, codecConfig.LargeMessageHandle.ClaimCheckStorageURI, changefeedID) - require.NoError(t, err) - defer claimCheckStorage.Close() - - err = claimCheckStorage.WriteMessage(ctx, largeMessage) - require.NoError(t, err) - - // claimCheckLocationMessage send to the kafka. - claimCheckLocationMessage, err := encoder.(codec.ClaimCheckLocationEncoder).NewClaimCheckLocationMessage(largeMessage) - require.NoError(t, err) - require.Empty(t, claimCheckLocationMessage.ClaimCheckFileName) - + claimCheckLocationMessage := messages[1] decoder, err := NewBatchDecoder(ctx, codecConfig, nil) require.NoError(t, err) err = decoder.AddKeyValue(claimCheckLocationMessage.Key, claimCheckLocationMessage.Value) @@ -497,9 +496,6 @@ func TestE2EClaimCheckMessage(t *testing.T) { require.Equal(t, messageType, model.MessageTypeRow) require.True(t, ok) - _, claimCheckFileName := filepath.Split(decoder.(*BatchDecoder).nextKey.ClaimCheckLocation) - require.Equal(t, largeMessage.ClaimCheckFileName, claimCheckFileName) - decodedLargeEvent, err := decoder.NextRowChangedEvent() require.NoError(t, err) diff --git a/pkg/sink/kafka/claimcheck/claim_check.go b/pkg/sink/kafka/claimcheck/claim_check.go index 69b831f3475..c34474d3cf6 100644 --- a/pkg/sink/kafka/claimcheck/claim_check.go +++ b/pkg/sink/kafka/claimcheck/claim_check.go @@ -30,6 +30,10 @@ import ( "go.uber.org/zap" ) +const ( + defaultTimeout = 5 * time.Minute +) + // ClaimCheck manage send message to the claim-check external storage. type ClaimCheck struct { storage storage.ExternalStorage @@ -44,15 +48,28 @@ type ClaimCheck struct { // New return a new ClaimCheck. func New(ctx context.Context, storageURI string, changefeedID model.ChangeFeedID) (*ClaimCheck, error) { - externalStorage, err := util.GetExternalStorageFromURI(ctx, storageURI) + log.Info("claim check enabled, start create the external storage", + zap.String("namespace", changefeedID.Namespace), + zap.String("changefeed", changefeedID.ID), + zap.String("storageURI", util.MaskSensitiveDataInURI(storageURI))) + + start := time.Now() + externalStorage, err := util.GetExternalStorageWithTimeout(ctx, storageURI, defaultTimeout) if err != nil { + log.Error("create external storage failed", + zap.String("namespace", changefeedID.Namespace), + zap.String("changefeed", changefeedID.ID), + zap.String("storageURI", util.MaskSensitiveDataInURI(storageURI)), + zap.Duration("duration", time.Since(start)), + zap.Error(err)) return nil, errors.Trace(err) } - log.Info("claim-check enabled", + log.Info("claim-check create the external storage success", zap.String("namespace", changefeedID.Namespace), zap.String("changefeed", changefeedID.ID), - zap.String("storageURI", storageURI)) + zap.String("storageURI", util.MaskSensitiveDataInURI(storageURI)), + zap.Duration("duration", time.Since(start))) return &ClaimCheck{ changefeedID: changefeedID, @@ -63,7 +80,7 @@ func New(ctx context.Context, storageURI string, changefeedID model.ChangeFeedID } // WriteMessage write message to the claim check external storage. -func (c *ClaimCheck) WriteMessage(ctx context.Context, message *common.Message) error { +func (c *ClaimCheck) WriteMessage(ctx context.Context, message *common.Message, fileName string) error { m := common.ClaimCheckMessage{ Key: message.Key, Value: message.Value, @@ -74,7 +91,7 @@ func (c *ClaimCheck) WriteMessage(ctx context.Context, message *common.Message) } start := time.Now() - err = c.storage.WriteFile(ctx, message.ClaimCheckFileName, data) + err = c.storage.WriteFile(ctx, fileName, data) if err != nil { return errors.Trace(err) } @@ -83,8 +100,13 @@ func (c *ClaimCheck) WriteMessage(ctx context.Context, message *common.Message) return nil } -// Close the claim check by clean up the metrics. -func (c *ClaimCheck) Close() { +// FileNameWithPrefix returns the file name with prefix, the full path. +func (c *ClaimCheck) FileNameWithPrefix(fileName string) string { + return filepath.Join(c.storage.URI(), fileName) +} + +// CleanMetrics the claim check by clean up the metrics. +func (c *ClaimCheck) CleanMetrics() { claimCheckSendMessageDuration.DeleteLabelValues(c.changefeedID.Namespace, c.changefeedID.ID) claimCheckSendMessageCount.DeleteLabelValues(c.changefeedID.Namespace, c.changefeedID.ID) } @@ -96,8 +118,3 @@ func (c *ClaimCheck) Close() { func NewFileName() string { return uuid.NewString() + ".json" } - -// FileNameWithPrefix returns the file name with prefix, the full path. -func FileNameWithPrefix(prefix, fileName string) string { - return filepath.Join(prefix, fileName) -} diff --git a/pkg/sink/mysql/config.go b/pkg/sink/mysql/config.go index f64c527f652..fadd9f7e17e 100644 --- a/pkg/sink/mysql/config.go +++ b/pkg/sink/mysql/config.go @@ -112,7 +112,6 @@ type Config struct { Timezone string TLS string ForceReplicate bool - EnableOldValue bool IsTiDB bool // IsTiDB is true if the downstream is TiDB // IsBDRModeSupported is true if the downstream is TiDB and write source is existed. @@ -198,7 +197,6 @@ func (c *Config) Apply( getBatchDMLEnable(urlParameter, &c.BatchDMLEnable) getMultiStmtEnable(urlParameter, &c.MultiStmtEnable) getCachePrepStmts(urlParameter, &c.CachePrepStmts) - c.EnableOldValue = replicaConfig.EnableOldValue c.ForceReplicate = replicaConfig.ForceReplicate c.SourceID = replicaConfig.Sink.TiDBSourceID diff --git a/pkg/sink/mysql/config_test.go b/pkg/sink/mysql/config_test.go index a210cad3ef1..2cf0c8391f3 100644 --- a/pkg/sink/mysql/config_test.go +++ b/pkg/sink/mysql/config_test.go @@ -189,7 +189,6 @@ func TestApplySinkURIParamsToConfig(t *testing.T) { expected.SafeMode = false expected.Timezone = `"UTC"` expected.tidbTxnMode = "pessimistic" - expected.EnableOldValue = true expected.CachePrepStmts = true uriStr := "mysql://127.0.0.1:3306/?worker-count=64&max-txn-row=20" + "&max-multi-update-row=80&max-multi-update-row-size=512" + diff --git a/pkg/sink/pulsar/config.go b/pkg/sink/pulsar/config.go index eaef867164f..6859fd2e561 100644 --- a/pkg/sink/pulsar/config.go +++ b/pkg/sink/pulsar/config.go @@ -51,6 +51,7 @@ func checkSinkURI(sinkURI *url.URL) error { } // NewPulsarConfig new pulsar config +// TODO(dongmen): make this method more concise. func NewPulsarConfig(sinkURI *url.URL, pulsarConfig *config.PulsarConfig) (*config.PulsarConfig, error) { c := &config.PulsarConfig{ ConnectionTimeout: toSec(defaultConnectionTimeout), @@ -64,20 +65,20 @@ func NewPulsarConfig(sinkURI *url.URL, pulsarConfig *config.PulsarConfig) (*conf return nil, err } - c.SetSinkURI(sinkURI) - c.SetBrokerURL(sinkURI.Scheme + "://" + sinkURI.Host) + c.SinkURI = sinkURI + c.BrokerURL = sinkURI.Scheme + "://" + sinkURI.Host if pulsarConfig == nil { log.L().Debug("new pulsar config", zap.Any("config", c)) return c, nil } - pulsarConfig.SetSinkURI(c.GetSinkURI()) + pulsarConfig.SinkURI = c.SinkURI if len(sinkURI.Scheme) == 0 || len(sinkURI.Host) == 0 { return nil, fmt.Errorf("BrokerURL is empty") } - pulsarConfig.SetBrokerURL(c.GetBrokerURL()) + pulsarConfig.BrokerURL = c.BrokerURL // merge default config if pulsarConfig.ConnectionTimeout == nil { diff --git a/pkg/sink/pulsar/config_test.go b/pkg/sink/pulsar/config_test.go index 205876e4923..8ff01c15777 100644 --- a/pkg/sink/pulsar/config_test.go +++ b/pkg/sink/pulsar/config_test.go @@ -76,7 +76,7 @@ func TestPulsarConfig(t *testing.T) { // If no error is expected, assert config values if !tt.wantErr { assert.Equal(t, config.CompressionType.Value(), pulsar.LZ4) - assert.Equal(t, config.GetBrokerURL(), "pulsar://127.0.0.1:6650") + assert.Equal(t, config.BrokerURL, "pulsar://127.0.0.1:6650") assert.Equal(t, config.ConnectionTimeout.Duration(), defaultConnectionTimeout*time.Second) assert.Equal(t, config.OperationTimeout.Duration(), 998*time.Second) assert.Equal(t, *config.BatchingMaxMessages, defaultBatchingMaxSize) @@ -93,7 +93,7 @@ func TestGetBrokerURL(t *testing.T) { replicaConfig := config.GetDefaultReplicaConfig() config, _ := NewPulsarConfig(sink, replicaConfig.Sink.PulsarConfig) - assert.Equal(t, config.GetBrokerURL(), "pulsar://localhost:6650") + assert.Equal(t, config.BrokerURL, "pulsar://localhost:6650") } func TestGetSinkURI(t *testing.T) { @@ -103,7 +103,7 @@ func TestGetSinkURI(t *testing.T) { replicaConfig := config.GetDefaultReplicaConfig() config, _ := NewPulsarConfig(sink, replicaConfig.Sink.PulsarConfig) - assert.Equal(t, config.GetSinkURI(), sink) + assert.Equal(t, config.SinkURI, sink) } func TestGetDefaultTopicName(t *testing.T) { diff --git a/pkg/sink/pulsar/factory.go b/pkg/sink/pulsar/factory.go index f27e639727a..979964203aa 100644 --- a/pkg/sink/pulsar/factory.go +++ b/pkg/sink/pulsar/factory.go @@ -14,8 +14,6 @@ package pulsar import ( - "fmt" - "github.com/apache/pulsar-client-go/pulsar" "github.com/apache/pulsar-client-go/pulsar/auth" "github.com/pingcap/log" @@ -30,8 +28,8 @@ type FactoryCreator func(config *config.PulsarConfig, changefeedID model.ChangeF // NewCreatorFactory returns a factory implemented based on kafka-go func NewCreatorFactory(config *config.PulsarConfig, changefeedID model.ChangeFeedID, sinkConfig *config.SinkConfig) (pulsar.Client, error) { - co := pulsar.ClientOptions{ - URL: config.GetBrokerURL(), + option := pulsar.ClientOptions{ + URL: config.BrokerURL, CustomMetricsLabels: map[string]string{ "changefeed": changefeedID.ID, "namespace": changefeedID.Namespace, @@ -40,10 +38,11 @@ func NewCreatorFactory(config *config.PulsarConfig, changefeedID model.ChangeFee OperationTimeout: config.OperationTimeout.Duration(), // add pulsar default metrics MetricsRegisterer: mq.GetMetricRegistry(), + Logger: NewPulsarLogger(), } var err error - co.Authentication, err = setupAuthentication(config) + option.Authentication, err = setupAuthentication(config) if err != nil { log.Error("setup pulsar authentication fail", zap.Error(err)) return nil, err @@ -54,13 +53,13 @@ func NewCreatorFactory(config *config.PulsarConfig, changefeedID model.ChangeFee sinkPulsar := sinkConfig.PulsarConfig if sinkPulsar.TLSCertificateFile != nil && sinkPulsar.TLSKeyFilePath != nil && sinkPulsar.TLSTrustCertsFilePath != nil { - co.TLSCertificateFile = *sinkPulsar.TLSCertificateFile - co.TLSKeyFilePath = *sinkPulsar.TLSKeyFilePath - co.TLSTrustCertsFilePath = *sinkPulsar.TLSTrustCertsFilePath + option.TLSCertificateFile = *sinkPulsar.TLSCertificateFile + option.TLSKeyFilePath = *sinkPulsar.TLSKeyFilePath + option.TLSTrustCertsFilePath = *sinkPulsar.TLSTrustCertsFilePath } } - pulsarClient, err := pulsar.NewClient(co) + pulsarClient, err := pulsar.NewClient(option) if err != nil { log.Error("cannot connect to pulsar", zap.Error(err)) return nil, err @@ -93,7 +92,8 @@ func setupAuthentication(config *config.PulsarConfig) (pulsar.Authentication, er if config.AuthTLSCertificatePath != nil && config.AuthTLSPrivateKeyPath != nil { return pulsar.NewAuthenticationTLS(*config.AuthTLSCertificatePath, *config.AuthTLSPrivateKeyPath), nil } - return nil, fmt.Errorf("no authentication method found") + log.Info("No authentication configured for pulsar client") + return nil, nil } // NewMockCreatorFactory returns a factory implemented based on kafka-go diff --git a/pkg/sink/pulsar/logger.go b/pkg/sink/pulsar/logger.go new file mode 100644 index 00000000000..835f0830199 --- /dev/null +++ b/pkg/sink/pulsar/logger.go @@ -0,0 +1,163 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package pulsar + +import ( + "github.com/apache/pulsar-client-go/pulsar/log" + plog "github.com/pingcap/log" + "go.uber.org/zap" +) + +// Logger wrapper cdc logger to adapt pulsar logger +type Logger struct { + zapLogger *zap.Logger +} + +// SubLogger sub +func (p *Logger) SubLogger(fields log.Fields) log.Logger { + subLogger := p.zapLogger + for k, v := range fields { + subLogger = subLogger.With(zap.Any(k, v)) + } + return &Logger{subLogger} +} + +// WithFields with fields +func (p *Logger) WithFields(fields log.Fields) log.Entry { + return &LoggerEntry{ + fields: fields, + logger: p.zapLogger, + } +} + +// WithField with field +func (p *Logger) WithField(name string, value interface{}) log.Entry { + return &LoggerEntry{ + fields: log.Fields{name: value}, + logger: p.zapLogger, + } +} + +// WithError error +func (p *Logger) WithError(err error) log.Entry { + return &LoggerEntry{ + fields: log.Fields{"error": err}, + logger: p.zapLogger, + } +} + +// Debug debug +func (p *Logger) Debug(args ...interface{}) { + p.zapLogger.Sugar().Debug(args...) +} + +// Info info +func (p *Logger) Info(args ...interface{}) { + p.zapLogger.Sugar().Info(args...) +} + +// Warn warn +func (p *Logger) Warn(args ...interface{}) { + p.zapLogger.Sugar().Warn(args...) +} + +// Error error +func (p *Logger) Error(args ...interface{}) { + p.zapLogger.Sugar().Error(args...) +} + +// Debugf debugf +func (p *Logger) Debugf(format string, args ...interface{}) { + p.zapLogger.Sugar().Debugf(format, args...) +} + +// Infof infof +func (p *Logger) Infof(format string, args ...interface{}) { + p.zapLogger.Sugar().Infof(format, args...) +} + +// Warnf warnf +func (p *Logger) Warnf(format string, args ...interface{}) { + p.zapLogger.Sugar().Warnf(format, args...) +} + +// Errorf errorf +func (p *Logger) Errorf(format string, args ...interface{}) { + p.zapLogger.Sugar().Errorf(format, args...) +} + +// NewPulsarLogger new pulsar logger +func NewPulsarLogger() *Logger { + return &Logger{ + zapLogger: plog.L(), + } +} + +// LoggerEntry pulsar logger entry +type LoggerEntry struct { + fields log.Fields + logger *zap.Logger +} + +// WithFields with fields +func (p *LoggerEntry) WithFields(fields log.Fields) log.Entry { + p.fields = fields + return p +} + +// WithField with field +func (p *LoggerEntry) WithField(name string, value interface{}) log.Entry { + p.fields[name] = value + return p +} + +// Debug debug +func (p *LoggerEntry) Debug(args ...interface{}) { + p.logger.Sugar().Debug(args...) +} + +// Info info +func (p *LoggerEntry) Info(args ...interface{}) { + p.logger.Sugar().Info(args...) +} + +// Warn warn +func (p *LoggerEntry) Warn(args ...interface{}) { + p.logger.Sugar().Warn(args...) +} + +// Error error +func (p *LoggerEntry) Error(args ...interface{}) { + p.logger.Sugar().Error(args...) +} + +// Debugf debugf +func (p *LoggerEntry) Debugf(format string, args ...interface{}) { + p.logger.Sugar().Debugf(format, args...) +} + +// Infof infof +func (p *LoggerEntry) Infof(format string, args ...interface{}) { + p.logger.Sugar().Infof(format, args...) +} + +// Warnf warnf +func (p *LoggerEntry) Warnf(format string, args ...interface{}) { + p.logger.Sugar().Warnf(format, args...) +} + +// Errorf errorf +func (p *LoggerEntry) Errorf(format string, args ...interface{}) { + p.logger.Sugar().Errorf(format, args...) +} diff --git a/pkg/sink/sink_type.go b/pkg/sink/sink_type.go index d257e87521e..382d8437689 100644 --- a/pkg/sink/sink_type.go +++ b/pkg/sink/sink_type.go @@ -13,6 +13,11 @@ package sink +import ( + "net/url" + "strings" +) + // Type is the type of sink. type Type int @@ -71,7 +76,8 @@ const ( // IsMQScheme returns true if the scheme belong to mq scheme. func IsMQScheme(scheme string) bool { - return scheme == KafkaScheme || scheme == KafkaSSLScheme + return scheme == KafkaScheme || scheme == KafkaSSLScheme || + scheme == PulsarScheme || scheme == PulsarSSLScheme } // IsMySQLCompatibleScheme returns true if the scheme is compatible with MySQL. @@ -85,3 +91,13 @@ func IsStorageScheme(scheme string) bool { return scheme == FileScheme || scheme == S3Scheme || scheme == GCSScheme || scheme == GSScheme || scheme == AzblobScheme || scheme == AzureScheme || scheme == CloudStorageNoopScheme } + +// IsPulsarScheme returns true if the scheme belong to pulsar scheme. +func IsPulsarScheme(scheme string) bool { + return scheme == PulsarScheme || scheme == PulsarSSLScheme +} + +// GetScheme returns the scheme of the url. +func GetScheme(url *url.URL) string { + return strings.ToLower(url.Scheme) +} diff --git a/pkg/upstream/manager.go b/pkg/upstream/manager.go index de5fb711619..d9c69077f4f 100644 --- a/pkg/upstream/manager.go +++ b/pkg/upstream/manager.go @@ -87,12 +87,16 @@ func NewManager4Test(pdClient pd.Client) *Manager { // AddDefaultUpstream add the default upstream func (m *Manager) AddDefaultUpstream(pdEndpoints []string, conf *security.Credential, + pdClient pd.Client, ) (*Upstream, error) { up := newUpstream(pdEndpoints, conf) + // use the pdClient pass from cdc server as the default upstream + // to reduce the creation times of pdClient to make cdc server more stable + up.isDefaultUpstream = true + up.PDClient = pdClient if err := m.initUpstreamFunc(m.ctx, up, m.gcServiceID); err != nil { return nil, err } - up.isDefaultUpstream = true m.defaultUpstream = up m.ups.Store(up.ID, up) log.Info("default upstream is added", zap.Uint64("id", up.ID)) diff --git a/pkg/upstream/manager_test.go b/pkg/upstream/manager_test.go index 870aa32f580..4bc544d9868 100644 --- a/pkg/upstream/manager_test.go +++ b/pkg/upstream/manager_test.go @@ -106,7 +106,8 @@ func TestAddDefaultUpstream(t *testing.T) { ) error { return errors.New("test") } - _, err := m.AddDefaultUpstream([]string{}, &security.Credential{}) + pdClient := &gc.MockPDClient{} + _, err := m.AddDefaultUpstream([]string{}, &security.Credential{}, pdClient) require.NotNil(t, err) up, err := m.GetDefaultUpstream() require.Nil(t, up) @@ -117,7 +118,7 @@ func TestAddDefaultUpstream(t *testing.T) { up.ID = uint64(2) return nil } - _, err = m.AddDefaultUpstream([]string{}, &security.Credential{}) + _, err = m.AddDefaultUpstream([]string{}, &security.Credential{}, pdClient) require.Nil(t, err) up, err = m.GetDefaultUpstream() require.NotNil(t, up) diff --git a/pkg/upstream/upstream.go b/pkg/upstream/upstream.go index fb576ddcf34..0c868791d06 100644 --- a/pkg/upstream/upstream.go +++ b/pkg/upstream/upstream.go @@ -122,28 +122,30 @@ func initUpstream(ctx context.Context, up *Upstream, gcServiceID string) error { } // init the tikv client tls global config initGlobalConfig(up.SecurityConfig) - - up.PDClient, err = pd.NewClientWithContext( - ctx, up.PdEndpoints, up.SecurityConfig.PDSecurityOption(), - // the default `timeout` is 3s, maybe too small if the pd is busy, - // set to 10s to avoid frequent timeout. - pd.WithCustomTimeoutOption(10*time.Second), - pd.WithGRPCDialOptions( - grpcTLSOption, - grpc.WithBlock(), - grpc.WithConnectParams(grpc.ConnectParams{ - Backoff: backoff.Config{ - BaseDelay: time.Second, - Multiplier: 1.1, - Jitter: 0.1, - MaxDelay: 3 * time.Second, - }, - MinConnectTimeout: 3 * time.Second, - }), - )) - if err != nil { - up.err.Store(err) - return errors.Trace(err) + // default upstream always use the pdClient pass from cdc server + if !up.isDefaultUpstream { + up.PDClient, err = pd.NewClientWithContext( + ctx, up.PdEndpoints, up.SecurityConfig.PDSecurityOption(), + // the default `timeout` is 3s, maybe too small if the pd is busy, + // set to 10s to avoid frequent timeout. + pd.WithCustomTimeoutOption(10*time.Second), + pd.WithGRPCDialOptions( + grpcTLSOption, + grpc.WithBlock(), + grpc.WithConnectParams(grpc.ConnectParams{ + Backoff: backoff.Config{ + BaseDelay: time.Second, + Multiplier: 1.1, + Jitter: 0.1, + MaxDelay: 3 * time.Second, + }, + MinConnectTimeout: 3 * time.Second, + }), + )) + if err != nil { + up.err.Store(err) + return errors.Trace(err) + } } clusterID := up.PDClient.GetClusterID(ctx) if up.ID != 0 && up.ID != clusterID { @@ -240,7 +242,9 @@ func (up *Upstream) Close() { } atomic.StoreInt32(&up.status, closing) - if up.PDClient != nil { + // should never close default upstream's pdClient here + // because it's shared by the cdc server + if up.PDClient != nil && !up.isDefaultUpstream { up.PDClient.Close() } diff --git a/pkg/util/comparison.go b/pkg/util/comparison.go new file mode 100644 index 00000000000..f89f216e474 --- /dev/null +++ b/pkg/util/comparison.go @@ -0,0 +1,33 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package util + +import "sort" + +// AreStringSlicesEquivalent checks if two string slices are equivalent. +// If the slices are of the same length and contain the same elements (but possibly in different order), the function returns true. +// Note: This function does modify the slices. Please be caution of this if you are using it. +func AreStringSlicesEquivalent(a, b []string) bool { + if len(a) != len(b) { + return false + } + sort.Strings(a) + sort.Strings(b) + for i, v := range a { + if v != b[i] { + return false + } + } + return true +} diff --git a/pkg/util/comparison_test.go b/pkg/util/comparison_test.go new file mode 100644 index 00000000000..49cf8edb655 --- /dev/null +++ b/pkg/util/comparison_test.go @@ -0,0 +1,61 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed 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, +// See the License for the specific language governing permissions and +// limitations under the License. +package util + +import ( + "testing" +) + +func TestAreStringSlicesEquivalent(t *testing.T) { + tests := []struct { + name string + a []string + b []string + want bool + }{ + { + name: "equal slices", + a: []string{"foo", "bar", "baz"}, + b: []string{"baz", "foo", "bar"}, + want: true, + }, + { + name: "different lengths", + a: []string{"foo", "bar", "baz"}, + b: []string{"foo", "bar"}, + want: false, + }, + { + name: "different elements", + a: []string{"foo", "bar", "baz"}, + b: []string{"qux", "quux", "corge"}, + want: false, + }, + { + name: "nil elements", + a: []string{}, + b: []string{}, + want: true, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + if got := AreStringSlicesEquivalent(tt.a, tt.b); got != tt.want { + t.Errorf("AreStringSlicesEquivalent() = %v, want %v", got, tt.want) + } + }) + } +} + +// END: j3d8f4b2j2p9 diff --git a/pkg/util/uri.go b/pkg/util/uri.go index dfe2b66f7b8..bf6753db4cb 100644 --- a/pkg/util/uri.go +++ b/pkg/util/uri.go @@ -80,3 +80,32 @@ func MaskSinkURI(uri string) (string, error) { } return uriParsed.Redacted(), nil } + +var sensitiveQueryParameterNames = []string{ + "password", + "sasl-password", + "access-key", + "secret-access-key", + "access_token", + "token", + "secret", + "passwd", + "pwd", +} + +// MaskSensitiveDataInURI returns an uri that sensitive infos has been masked. +func MaskSensitiveDataInURI(uri string) string { + uriParsed, err := url.Parse(uri) + if err != nil { + log.Error("failed to parse sink URI", zap.Error(err)) + return "" + } + queries := uriParsed.Query() + for _, secretKey := range sensitiveQueryParameterNames { + if queries.Has(secretKey) { + queries.Set(secretKey, "xxxxx") + } + } + uriParsed.RawQuery = queries.Encode() + return uriParsed.Redacted() +} diff --git a/pkg/util/uri_test.go b/pkg/util/uri_test.go index 7cafebc9ef6..97e16b511f0 100644 --- a/pkg/util/uri_test.go +++ b/pkg/util/uri_test.go @@ -88,3 +88,37 @@ func TestMaskSinkURI(t *testing.T) { require.Equal(t, tt.masked, maskedURI) } } + +func TestMaskSensitiveDataInURI(t *testing.T) { + tests := []struct { + uri string + masked string + }{ + { + "mysql://root:123456@127.0.0.1:3306/?time-zone=c", + "mysql://root:xxxxx@127.0.0.1:3306/?time-zone=c", + }, + { + "", + "", + }, + { + "abc", + "abc", + }, + } + for _, q := range sensitiveQueryParameterNames { + tests = append(tests, struct { + uri string + masked string + }{ + "kafka://127.0.0.1:9093/cdc?" + q + "=verysecure", + "kafka://127.0.0.1:9093/cdc?" + q + "=xxxxx", + }) + } + + for _, tt := range tests { + maskedURI := MaskSensitiveDataInURI(tt.uri) + require.Equal(t, tt.masked, maskedURI) + } +} diff --git a/pkg/version/check.go b/pkg/version/check.go index 2be6566f2a3..1399c0734e4 100644 --- a/pkg/version/check.go +++ b/pkg/version/check.go @@ -239,8 +239,8 @@ type TiCDCClusterVersion struct { *semver.Version } -// ShouldEnableOldValueByDefault returns whether old value should be enabled by default -func (v *TiCDCClusterVersion) ShouldEnableOldValueByDefault() bool { +// LessThan500RC returns true if th cluster version is less than 5.0.0-rc +func (v *TiCDCClusterVersion) LessThan500RC() bool { // we assume the unknown version to be the latest version return v.Version == nil || !v.LessThan(*semver.New("5.0.0-rc")) } diff --git a/pkg/version/check_test.go b/pkg/version/check_test.go index fad6a1587b8..ba3eef8a916 100644 --- a/pkg/version/check_test.go +++ b/pkg/version/check_test.go @@ -357,46 +357,46 @@ func TestTiCDCClusterVersionFeaturesCompatible(t *testing.T) { t.Parallel() ver := TiCDCClusterVersion{semver.New("4.0.10")} require.Equal(t, ver.ShouldEnableUnifiedSorterByDefault(), false) - require.Equal(t, ver.ShouldEnableOldValueByDefault(), false) + require.Equal(t, ver.LessThan500RC(), false) ver = TiCDCClusterVersion{semver.New("4.0.12")} require.Equal(t, ver.ShouldEnableUnifiedSorterByDefault(), false) - require.Equal(t, ver.ShouldEnableOldValueByDefault(), false) + require.Equal(t, ver.LessThan500RC(), false) ver = TiCDCClusterVersion{semver.New("4.0.13")} require.Equal(t, ver.ShouldEnableUnifiedSorterByDefault(), true) - require.Equal(t, ver.ShouldEnableOldValueByDefault(), false) + require.Equal(t, ver.LessThan500RC(), false) ver = TiCDCClusterVersion{semver.New("4.0.13-hotfix")} require.Equal(t, ver.ShouldEnableUnifiedSorterByDefault(), true) - require.Equal(t, ver.ShouldEnableOldValueByDefault(), false) + require.Equal(t, ver.LessThan500RC(), false) ver = TiCDCClusterVersion{semver.New("4.0.14")} require.Equal(t, ver.ShouldEnableUnifiedSorterByDefault(), true) - require.Equal(t, ver.ShouldEnableOldValueByDefault(), false) + require.Equal(t, ver.LessThan500RC(), false) ver = TiCDCClusterVersion{semver.New("5.0.0-rc")} require.Equal(t, ver.ShouldEnableUnifiedSorterByDefault(), false) - require.Equal(t, ver.ShouldEnableOldValueByDefault(), true) + require.Equal(t, ver.LessThan500RC(), true) ver = TiCDCClusterVersion{semver.New("5.0.0")} require.Equal(t, ver.ShouldEnableUnifiedSorterByDefault(), true) - require.Equal(t, ver.ShouldEnableOldValueByDefault(), true) + require.Equal(t, ver.LessThan500RC(), true) ver = TiCDCClusterVersion{semver.New("5.1.0")} require.Equal(t, ver.ShouldEnableUnifiedSorterByDefault(), true) - require.Equal(t, ver.ShouldEnableOldValueByDefault(), true) + require.Equal(t, ver.LessThan500RC(), true) ver = TiCDCClusterVersion{semver.New("5.2.0-alpha")} require.Equal(t, ver.ShouldEnableUnifiedSorterByDefault(), true) - require.Equal(t, ver.ShouldEnableOldValueByDefault(), true) + require.Equal(t, ver.LessThan500RC(), true) ver = TiCDCClusterVersion{semver.New("5.2.0-master")} require.Equal(t, ver.ShouldEnableUnifiedSorterByDefault(), true) - require.Equal(t, ver.ShouldEnableOldValueByDefault(), true) + require.Equal(t, ver.LessThan500RC(), true) require.Equal(t, ticdcClusterVersionUnknown.ShouldEnableUnifiedSorterByDefault(), true) - require.Equal(t, ticdcClusterVersionUnknown.ShouldEnableOldValueByDefault(), true) + require.Equal(t, ticdcClusterVersionUnknown.LessThan500RC(), true) } func TestCheckPDVersionError(t *testing.T) { diff --git a/pkg/version/creator_version_gate.go b/pkg/version/creator_version_gate.go index 31ab5bea9a3..0480f65f47f 100644 --- a/pkg/version/creator_version_gate.go +++ b/pkg/version/creator_version_gate.go @@ -78,10 +78,7 @@ func (g *CreatorVersionGate) ChangefeedAcceptUnknownProtocols() bool { return creatorVersion.LessThan(changefeedAcceptUnknownProtocolsVersion) } -var ( - changefeedAcceptProtocolInMysqlSinURI = *semver.New("6.1.1") - changefeedAdjustEnableOldValueByProtocol = *semver.New("7.2.0") -) +var changefeedAcceptProtocolInMysqlSinURI = *semver.New("6.1.1") // ChangefeedAcceptProtocolInMysqlSinURI determines whether to accept // protocol in mysql sink uri or configure based on the creator's version. @@ -106,14 +103,3 @@ func (g *CreatorVersionGate) ChangefeedInheritSchedulerConfigFromV66() bool { creatorVersion := semver.New(SanitizeVersion(g.version)) return creatorVersion.Major == 6 && creatorVersion.Minor == 6 } - -// ChangefeedAdjustEnableOldValueByProtocol determines whether to adjust -// the `enable-old-value` configuration by the using encoding protocol. -func (g *CreatorVersionGate) ChangefeedAdjustEnableOldValueByProtocol() bool { - if g.version == "" { - return true - } - - creatorVersion := semver.New(SanitizeVersion(g.version)) - return creatorVersion.LessThan(changefeedAdjustEnableOldValueByProtocol) -} diff --git a/tests/integration_tests/api_v2/cases.go b/tests/integration_tests/api_v2/cases.go index 3156e0bf565..f62cf45e011 100644 --- a/tests/integration_tests/api_v2/cases.go +++ b/tests/integration_tests/api_v2/cases.go @@ -28,7 +28,6 @@ import ( var customReplicaConfig = &ReplicaConfig{ MemoryQuota: 1123450, CaseSensitive: false, - EnableOldValue: false, ForceReplicate: false, IgnoreIneligibleTable: false, CheckGCSafePoint: false, @@ -81,7 +80,6 @@ var customReplicaConfig = &ReplicaConfig{ var defaultReplicaConfig = &ReplicaConfig{ MemoryQuota: 1024 * 1024 * 1024, CaseSensitive: true, - EnableOldValue: true, CheckGCSafePoint: true, Filter: &FilterConfig{ Rules: []string{"*.*"}, diff --git a/tests/integration_tests/api_v2/model.go b/tests/integration_tests/api_v2/model.go index 31299d71d20..7ef56dffc9e 100644 --- a/tests/integration_tests/api_v2/model.go +++ b/tests/integration_tests/api_v2/model.go @@ -165,7 +165,6 @@ func (d *JSONDuration) UnmarshalJSON(b []byte) error { type ReplicaConfig struct { MemoryQuota uint64 `json:"memory_quota"` CaseSensitive bool `json:"case_sensitive"` - EnableOldValue bool `json:"enable_old_value"` ForceReplicate bool `json:"force_replicate"` IgnoreIneligibleTable bool `json:"ignore_ineligible_table"` CheckGCSafePoint bool `json:"check_gc_safe_point"` diff --git a/tests/integration_tests/force_replicate_table/conf/changefeed.toml b/tests/integration_tests/force_replicate_table/conf/changefeed.toml index 6be302ceda0..dadcdb1a657 100644 --- a/tests/integration_tests/force_replicate_table/conf/changefeed.toml +++ b/tests/integration_tests/force_replicate_table/conf/changefeed.toml @@ -1,2 +1 @@ -enable-old-value = true force-replicate = true diff --git a/tests/integration_tests/hang_sink_suicide/run.sh b/tests/integration_tests/hang_sink_suicide/run.sh index 3489df74e05..e4e663cb975 100644 --- a/tests/integration_tests/hang_sink_suicide/run.sh +++ b/tests/integration_tests/hang_sink_suicide/run.sh @@ -41,6 +41,7 @@ function run() { } trap stop_tidb_cluster EXIT -run $* -check_logs $WORK_DIR +# TODO: update the case to use kafka sink instead of mysql sink. +# run $* +# check_logs $WORK_DIR echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" diff --git a/tests/integration_tests/http_api/util/test_case.py b/tests/integration_tests/http_api/util/test_case.py index 605aa2e9704..756758ea885 100644 --- a/tests/integration_tests/http_api/util/test_case.py +++ b/tests/integration_tests/http_api/util/test_case.py @@ -1,6 +1,7 @@ import sys import os import requests as rq +from requests.exceptions import RequestException import time import json @@ -24,6 +25,26 @@ SINK_URI="mysql://normal:%s@127.0.0.1:3306/" % ENPASSWORD physicalShiftBits = 18 + +def requests_get_with_retry(url, max_retries=RETRY_TIME, delay_seconds=1): + """ + requests get with retry + + :param url: request url + :param max_retries: max retry times + :param delay_seconds: retry delay seconds + :return: when success, return response, else return None + """ + for retry in range(max_retries): + try: + response = rq.get(url) + if response.status_code == 200 or response.status_code == 202: + return response + except RequestException as e: + print(f"request fails {retry + 1}/{max_retries} time retry...") + time.sleep(delay_seconds) + return None + # we should write some SQLs in the run.sh after call create_changefeed def create_changefeed(sink_uri): url = BASE_URL1+"/changefeeds" @@ -245,7 +266,7 @@ def resign_owner(): def list_capture(): url = BASE_URL0 + "/captures" - resp = rq.get(url) + resp = requests_get_with_retry(url) assert resp.status_code == rq.codes.ok print("pass test: list captures") @@ -253,7 +274,7 @@ def list_capture(): def list_processor(): url = BASE_URL0 + "/processors" - resp = rq.get(url) + resp = requests_get_with_retry(url) assert resp.status_code == rq.codes.ok print("pass test: list processors") @@ -262,17 +283,16 @@ def list_processor(): def get_processor(): # list processor to get changefeed_id and capture_id base_url = BASE_URL0 + "/processors" - resp = rq.get(base_url) + resp = requests_get_with_retry(base_url) assert resp.status_code == rq.codes.ok data = resp.json()[0] time.sleep(2) url = base_url + "/" + data["changefeed_id"] + "/" + data["capture_id"] - resp = rq.get(url) + resp = requests_get_with_retry(url) # print error message for debug if (resp.status_code != rq.codes.ok): print("request url", url) print("response status code:", resp.status_code) - print("response body:", resp.text()) assert resp.status_code == rq.codes.ok # test capture_id error and cdc server no panic @@ -297,7 +317,7 @@ def check_health(): def get_status(): url = BASE_URL0 + "/status" - resp = rq.get(url) + resp = requests_get_with_retry(url) assert resp.status_code == rq.codes.ok assert resp.json()["is_owner"] diff --git a/tests/integration_tests/http_proxies/run-proxy.go b/tests/integration_tests/http_proxies/run-proxy.go index e8bc936c826..b7d244426f1 100644 --- a/tests/integration_tests/http_proxies/run-proxy.go +++ b/tests/integration_tests/http_proxies/run-proxy.go @@ -21,11 +21,19 @@ import ( "github.com/pingcap/log" "go.uber.org/zap" "google.golang.org/grpc" + "google.golang.org/grpc/metadata" ) func main() { + defer func() { + fmt.Println("proxy stopped") + }() + grpc_proxy.RegisterDefaultFlags() flag.Parse() + + log.Info("starting proxy", zap.Any("flags", flag.Args())) + proxy, err := grpc_proxy.New( grpc_proxy.WithInterceptor(intercept), grpc_proxy.DefaultFlags(), @@ -37,9 +45,19 @@ func main() { if err != nil { log.Fatal("failed to start proxy", zap.Error(err)) } + fmt.Println("proxy started") } func intercept(srv interface{}, ss grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error { fmt.Println(info.FullMethod) - return handler(srv, ss) + err := handler(srv, ss) + if err != nil { + md, ok := metadata.FromIncomingContext(ss.Context()) + log.Error("failed to handle stream", + zap.String("method", info.FullMethod), + zap.Bool("ok", ok), + zap.Any("metadata", md), + zap.Error(err)) + } + return err } diff --git a/tests/integration_tests/http_proxies/run.sh b/tests/integration_tests/http_proxies/run.sh index 7ef620df91f..3be2ef77dbe 100644 --- a/tests/integration_tests/http_proxies/run.sh +++ b/tests/integration_tests/http_proxies/run.sh @@ -23,9 +23,11 @@ export UP_TIDB_HOST=$lan_addr \ proxy_pid="" proxy_port=$(shuf -i 10081-20081 -n1) function start_proxy() { - echo "dumpling grpc packet to $WORK_DIR/packets.dump..." - GO111MODULE=on WORK_DIR=$WORK_DIR go run $CUR/run-proxy.go --port=$proxy_port >$WORK_DIR/packets.dump & + echo "dumpling grpc packet to $WORK_DIR/test_proxy.log..." + GO111MODULE=on WORK_DIR=$WORK_DIR go run $CUR/run-proxy.go --port=$proxy_port >$WORK_DIR/test_proxy.log & proxy_pid=$! + echo "proxy port: $proxy_port" + echo "proxy pid: $proxy_pid" } function stop_proxy() { @@ -55,21 +57,25 @@ function prepare() { sleep 5 export http_proxy=http://127.0.0.1:$proxy_port export https_proxy=http://127.0.0.1:$proxy_port + echo "try to connect pd cluster via proxy, pd addr: $UP_PD_HOST_1:2379" ensure 10 curl http://$UP_PD_HOST_1:2379/ - echo started proxy at $proxy_pid + echo started proxy pid: $proxy_pid + echo started proxy at port: $proxy_port cd $WORK_DIR start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) + echo "query start ts: $start_ts" run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + echo started cdc server successfully SINK_URI="blackhole:///" run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" } function check() { - services=($(cat $WORK_DIR/packets.dump | xargs -L1 dirname | sort | uniq)) + services=($(cat $WORK_DIR/test_proxy.log | xargs -L1 dirname | sort | uniq)) service_type_count=${#services[@]} echo "captured services: " echo ${services[@]} diff --git a/tests/integration_tests/multi_changefeed/conf/changefeed1.toml b/tests/integration_tests/multi_changefeed/conf/changefeed1.toml deleted file mode 100644 index 4293a79e451..00000000000 --- a/tests/integration_tests/multi_changefeed/conf/changefeed1.toml +++ /dev/null @@ -1 +0,0 @@ -enable-old-value = true diff --git a/tests/integration_tests/multi_changefeed/conf/changefeed2.toml b/tests/integration_tests/multi_changefeed/conf/changefeed2.toml deleted file mode 100644 index d3ddcb27ea1..00000000000 --- a/tests/integration_tests/multi_changefeed/conf/changefeed2.toml +++ /dev/null @@ -1 +0,0 @@ -enable-old-value = false diff --git a/tests/integration_tests/multi_changefeed/run.sh b/tests/integration_tests/multi_changefeed/run.sh deleted file mode 100755 index 983d8b0c5c4..00000000000 --- a/tests/integration_tests/multi_changefeed/run.sh +++ /dev/null @@ -1,76 +0,0 @@ -#!/bin/bash - -set -eu - -CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -source $CUR/../_utils/test_prepare -WORK_DIR=$OUT_DIR/$TEST_NAME -CDC_BINARY=cdc.test -SINK_TYPE=$1 - -MAX_RETRIES=10 - -function check_old_value_enabled() { - row_logs=$(grep "BlackHoleSink: WriteEvents" "$1/cdc.log") - echo $row_logs - - # check update rows - # check if exist a row include `column` and `pre-column` - # When old value is turned on, we will have both column and pre-column in the update. - # So here we have 2 (pre val) and 3 (new val). - update_with_old_value_count=$(grep "BlackHoleSink: WriteEvents" "$1/cdc.log" | grep 'pre\-columns\\\":\[' | grep '\"columns\\\":\[' | grep 'value\\\":2' | grep -c 'value\\\":3') - if [[ "$update_with_old_value_count" -ne 1 ]]; then - echo "can't found update row with old value" - exit 1 - fi - - # check if exist a update row without `pre-column` - # When old value is turned off, we only have the column in the update. - # So here we only have 3 (new val). - update_without_old_value_count=$(grep "BlackHoleSink: WriteEvents" "$1/cdc.log" | grep 'pre\-columns\\\":null' | grep -c 'value\\\":3') - if [[ "$update_without_old_value_count" -ne 1 ]]; then - echo "can't found update row without old value" - exit 1 - fi - - # check delete rows - # check if exist a delete row with a complete `pre-column` - # When old value is turned on, the pre-column in our delete will include all the columns. - # So here we have 1 (id) and 3 (val). - delete_with_old_value_count=$(grep "BlackHoleSink: WriteEvents" "$1/cdc.log" | grep 'pre\-columns\\\":\[' | grep 'columns\\\":null' | grep 'value\\\":1' | grep -c 'value\\\":3') - if [[ "$delete_with_old_value_count" -ne 2 ]]; then - echo "can't found delete row with old value, not 2 found" - exit 1 - fi -} - -export -f check_old_value_enabled - -function run() { - if [ "$SINK_TYPE" != "mysql" ]; then - return - fi - - rm -rf $WORK_DIR && mkdir -p $WORK_DIR - start_tidb_cluster --workdir $WORK_DIR - cd $WORK_DIR - - pd_addr="http://$UP_PD_HOST_1:$UP_PD_PORT_1" - SINK_URI="blackhole://" - - run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8300" --pd $pd_addr - cdc cli changefeed create --pd=$pd_addr --sink-uri="$SINK_URI" -c "old-value-cf" --config="$CUR/conf/changefeed1.toml" - cdc cli changefeed create --pd=$pd_addr --sink-uri="$SINK_URI" -c "no-old-value-cf" --config="$CUR/conf/changefeed2.toml" - run_sql "CREATE DATABASE multi_changefeed;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "CREATE table multi_changefeed.t1 (id int primary key, val int);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "INSERT INTO multi_changefeed.t1 VALUES (1,2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "UPDATE multi_changefeed.t1 SET val = 3;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "DELETE FROM multi_changefeed.t1;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - - ensure $MAX_RETRIES check_old_value_enabled $WORK_DIR - cleanup_process $CDC_BINARY -} - -trap stop_tidb_cluster EXIT -run $* -echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" diff --git a/tests/integration_tests/new_ci_collation_with_old_value/conf/diff_config.toml b/tests/integration_tests/new_ci_collation/conf/diff_config.toml similarity index 70% rename from tests/integration_tests/new_ci_collation_with_old_value/conf/diff_config.toml rename to tests/integration_tests/new_ci_collation/conf/diff_config.toml index 32b8417e91f..e249e6df937 100644 --- a/tests/integration_tests/new_ci_collation_with_old_value/conf/diff_config.toml +++ b/tests/integration_tests/new_ci_collation/conf/diff_config.toml @@ -7,13 +7,13 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/tidb_cdc_test/new_ci_collation_with_old_value/sync_diff/output" + output-dir = "/tmp/tidb_cdc_test/new_ci_collation/sync_diff/output" source-instances = ["mysql1"] target-instance = "tidb0" - target-check-tables = ["new_ci_collation_with_old_value_test.?*"] + target-check-tables = ["new_ci_collation_test.?*"] [data-sources] [data-sources.mysql1] diff --git a/tests/integration_tests/new_ci_collation_with_old_value/conf/tidb_config.toml b/tests/integration_tests/new_ci_collation/conf/tidb_config.toml similarity index 100% rename from tests/integration_tests/new_ci_collation_with_old_value/conf/tidb_config.toml rename to tests/integration_tests/new_ci_collation/conf/tidb_config.toml diff --git a/tests/integration_tests/new_ci_collation_with_old_value/data/test1.sql b/tests/integration_tests/new_ci_collation/data/test1.sql similarity index 90% rename from tests/integration_tests/new_ci_collation_with_old_value/data/test1.sql rename to tests/integration_tests/new_ci_collation/data/test1.sql index c757bb228b7..a689dbae138 100644 --- a/tests/integration_tests/new_ci_collation_with_old_value/data/test1.sql +++ b/tests/integration_tests/new_ci_collation/data/test1.sql @@ -1,6 +1,6 @@ -drop database if exists `new_ci_collation_with_old_value_test`; -create database `new_ci_collation_with_old_value_test`; -use `new_ci_collation_with_old_value_test`; +drop database if exists `new_ci_collation_test`; +create database `new_ci_collation_test`; +use `new_ci_collation_test`; CREATE TABLE t1 ( a varchar(20) charset utf8mb4 collate utf8mb4_general_ci primary key, diff --git a/tests/integration_tests/new_ci_collation_with_old_value/data/test2.sql b/tests/integration_tests/new_ci_collation/data/test2.sql similarity index 64% rename from tests/integration_tests/new_ci_collation_with_old_value/data/test2.sql rename to tests/integration_tests/new_ci_collation/data/test2.sql index a977148f12e..40a1a6b0331 100644 --- a/tests/integration_tests/new_ci_collation_with_old_value/data/test2.sql +++ b/tests/integration_tests/new_ci_collation/data/test2.sql @@ -1,4 +1,4 @@ -use `new_ci_collation_with_old_value_test`; +use `new_ci_collation_test`; delete from t1; delete from t2; delete from t3; diff --git a/tests/integration_tests/new_ci_collation_without_old_value/run.sh b/tests/integration_tests/new_ci_collation/run.sh similarity index 85% rename from tests/integration_tests/new_ci_collation_without_old_value/run.sh rename to tests/integration_tests/new_ci_collation/run.sh index d3c868fa13d..80548b1362d 100755 --- a/tests/integration_tests/new_ci_collation_without_old_value/run.sh +++ b/tests/integration_tests/new_ci_collation/run.sh @@ -25,21 +25,23 @@ function run() { run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY - TOPIC_NAME="ticdc-new_ci_collation_without_old_value-test-$RANDOM" + TOPIC_NAME="ticdc-new_ci_collation-test-$RANDOM" case $SINK_TYPE in - kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=open-protocol&partition-num=4&kafka-version=${KAFKA_VERSION}" ;; + kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=open-protocol&partition-num=4&kafka-version=${KAFKA_VERSION}&max-message-bytes=10485760" ;; storage) SINK_URI="file://$WORK_DIR/storage_test/$TOPIC_NAME?protocol=canal-json&enable-tidb-extension=true" ;; *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/?safe-mode=true" ;; esac - cdc cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --config $CUR/conf/changefeed.toml + + cdc cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" + case $SINK_TYPE in kafka) run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=open-protocol&partition-num=4&version=${KAFKA_VERSION}&max-message-bytes=10485760" ;; - storage) run_storage_consumer $WORK_DIR $SINK_URI "" "" ;; + storage) run_storage_consumer $WORK_DIR $SINK_URI "" ;; esac run_sql_file $CUR/data/test1.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} for i in $(seq 1 5); do - table="new_ci_collation_without_old_value_test.t$i" + table="new_ci_collation_test.t$i" check_table_exists $table ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} done check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml diff --git a/tests/integration_tests/new_ci_collation_with_old_value/conf/changefeed.toml b/tests/integration_tests/new_ci_collation_with_old_value/conf/changefeed.toml deleted file mode 100644 index 4293a79e451..00000000000 --- a/tests/integration_tests/new_ci_collation_with_old_value/conf/changefeed.toml +++ /dev/null @@ -1 +0,0 @@ -enable-old-value = true diff --git a/tests/integration_tests/new_ci_collation_with_old_value/run.sh b/tests/integration_tests/new_ci_collation_with_old_value/run.sh deleted file mode 100755 index a246b518a0f..00000000000 --- a/tests/integration_tests/new_ci_collation_with_old_value/run.sh +++ /dev/null @@ -1,57 +0,0 @@ -#!/bin/bash - -set -eu - -CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -source $CUR/../_utils/test_prepare -WORK_DIR=$OUT_DIR/$TEST_NAME -CDC_BINARY=cdc.test -SINK_TYPE=$1 - -function run() { - # storage is not supported yet. - if [ "$SINK_TYPE" == "storage" ]; then - return - fi - - rm -rf $WORK_DIR && mkdir -p $WORK_DIR - - start_tidb_cluster --workdir $WORK_DIR --tidb-config $CUR/conf/tidb_config.toml - - cd $WORK_DIR - - # record tso before we create tables to skip the system table DDLs - start_ts=$(cdc cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) - - run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY - - TOPIC_NAME="ticdc-new_ci_collation_with_old_value-test-$RANDOM" - case $SINK_TYPE in - kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=open-protocol&partition-num=4&kafka-version=${KAFKA_VERSION}&max-message-bytes=10485760" ;; - storage) SINK_URI="file://$WORK_DIR/storage_test/$TOPIC_NAME?protocol=canal-json&enable-tidb-extension=true" ;; - *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/?safe-mode=true" ;; - esac - - cdc cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --config $CUR/conf/changefeed.toml - - case $SINK_TYPE in - kafka) run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=open-protocol&partition-num=4&version=${KAFKA_VERSION}&max-message-bytes=10485760" $CUR/conf/changefeed.toml ;; - storage) run_storage_consumer $WORK_DIR $SINK_URI $CUR/conf/changefeed.toml "" ;; - esac - - run_sql_file $CUR/data/test1.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} - for i in $(seq 1 5); do - table="new_ci_collation_with_old_value_test.t$i" - check_table_exists $table ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - done - check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml - run_sql_file $CUR/data/test2.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml - - cleanup_process $CDC_BINARY -} - -trap stop_tidb_cluster EXIT -run $* -check_logs $WORK_DIR -echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" diff --git a/tests/integration_tests/new_ci_collation_without_old_value/conf/changefeed.toml b/tests/integration_tests/new_ci_collation_without_old_value/conf/changefeed.toml deleted file mode 100644 index d3ddcb27ea1..00000000000 --- a/tests/integration_tests/new_ci_collation_without_old_value/conf/changefeed.toml +++ /dev/null @@ -1 +0,0 @@ -enable-old-value = false diff --git a/tests/integration_tests/new_ci_collation_without_old_value/conf/diff_config.toml b/tests/integration_tests/new_ci_collation_without_old_value/conf/diff_config.toml deleted file mode 100644 index 412de80d2a2..00000000000 --- a/tests/integration_tests/new_ci_collation_without_old_value/conf/diff_config.toml +++ /dev/null @@ -1,29 +0,0 @@ -# diff Configuration. - -check-thread-count = 4 - -export-fix-sql = true - -check-struct-only = false - -[task] - output-dir = "/tmp/tidb_cdc_test/new_ci_collation_without_old_value/sync_diff/output" - - source-instances = ["mysql1"] - - target-instance = "tidb0" - - target-check-tables = ["new_ci_collation_without_old_value_test.?*"] - -[data-sources] -[data-sources.mysql1] - host = "127.0.0.1" - port = 4000 - user = "root" - password = "" - -[data-sources.tidb0] - host = "127.0.0.1" - port = 3306 - user = "root" - password = "" diff --git a/tests/integration_tests/new_ci_collation_without_old_value/conf/tidb_config.toml b/tests/integration_tests/new_ci_collation_without_old_value/conf/tidb_config.toml deleted file mode 100644 index bc23af7783a..00000000000 --- a/tests/integration_tests/new_ci_collation_without_old_value/conf/tidb_config.toml +++ /dev/null @@ -1,2 +0,0 @@ -alter-primary-key = true -new_collations_enabled_on_first_bootstrap = true diff --git a/tests/integration_tests/new_ci_collation_without_old_value/data/test1.sql b/tests/integration_tests/new_ci_collation_without_old_value/data/test1.sql deleted file mode 100644 index 7332a3c7623..00000000000 --- a/tests/integration_tests/new_ci_collation_without_old_value/data/test1.sql +++ /dev/null @@ -1,56 +0,0 @@ -drop database if exists `new_ci_collation_without_old_value_test`; -create database `new_ci_collation_without_old_value_test`; -use `new_ci_collation_without_old_value_test`; - -CREATE TABLE t1 ( - a varchar(20) charset utf8mb4 collate utf8mb4_general_ci primary key, - b int default 10 -); - -CREATE TABLE t2 ( - a varchar(10) charset utf8 collate utf8_general_ci, primary key(a), - b int default 10 -); - -CREATE TABLE t3 ( - id int primary key auto_increment, - a varchar(20) charset utf8mb4 collate utf8mb4_general_ci, - b int default 10 -); - -CREATE TABLE t4 ( - a int primary key, - b varchar(10) charset utf8mb4 collate utf8mb4_general_ci, - c varchar(10) charset utf8 collate utf8_general_ci, - d varchar(10) not null, - unique key d(d) -); - -CREATE TABLE t5 ( - a varchar(10) charset utf8mb4 collate utf8mb4_bin, primary key(a), - b int default 10 -); - - -insert into t1 (a) values ('A'),(' A'),('A\t'),('b'),('bA'),('bac'),('ab'); -insert into t1 (a) values ('😉'); - -insert into t2 (a) values ('A'),(' A'),('A\t'),('b'),('bA'),('bac'),('ab'); - -insert into t3() values(); -insert into t3 (a) values ('A'),('A '),('A '),(' A'),('A\t'),('A\t '); -insert into t3 (a) values ('a'),('a '),('a '),(' a'),('a\t'),('a\t '); -insert into t3 (a) values ('B'),('B '),('B '),(' B'),('B\t'),('B\t '); -insert into t3 (a) values ('b'),('b '),('b '),(' b'),('b\t'),('b\t '); - -insert into t4 values (1,'A','A','1'),(2,'a\t','a\t','2'),(3,'ab','ab','3'),(4,'abc','abc','4'); -insert into t4(a, d) values(5, "a"); - -insert into t5 (a) values ('😉'); -insert into t5 (a) values ('a'),('A'),(' a'),(' A'),('a\t'),('ab'),('Ab'); - -update t1 set b = b + 1; -update t2 set b = 13; -update t3 set b = 11 where a > 'A'; -drop index `primary` on t4; -update t5 set b = 12; diff --git a/tests/integration_tests/new_ci_collation_without_old_value/data/test2.sql b/tests/integration_tests/new_ci_collation_without_old_value/data/test2.sql deleted file mode 100644 index dba8001ba91..00000000000 --- a/tests/integration_tests/new_ci_collation_without_old_value/data/test2.sql +++ /dev/null @@ -1,6 +0,0 @@ -use `new_ci_collation_without_old_value_test`; -delete from t1; -delete from t2; -delete from t3; -delete from t4; -delete from t5; diff --git a/tests/integration_tests/run_group.sh b/tests/integration_tests/run_group.sh index 80e222902e3..8f4f90e9a75 100755 --- a/tests/integration_tests/run_group.sh +++ b/tests/integration_tests/run_group.sh @@ -44,16 +44,16 @@ groups=( ["G10"]='default_value simple cdc_server_tips event_filter' ["G11"]='resolve_lock move_table autorandom generate_column' ["G12"]='many_pk_or_uk capture_session_done_during_task ddl_attributes' - ["G13"]='tiflash new_ci_collation_without_old_value region_merge common_1' + ["G13"]='tiflash region_merge common_1' ["G14"]='big_txn changefeed_finish force_replicate_table' - ["G15"]='new_ci_collation_with_old_value batch_add_table multi_rocks' + ["G15"]='new_ci_collation batch_add_table multi_rocks' # currently G16 is not running in kafka pipeline ["G16"]='owner_resign processor_etcd_worker_delay sink_hang' ["G17"]='clustered_index processor_resolved_ts_fallback' # only run the following tests in mysql pipeline ["G18"]='availability http_proxies sequence' ["G19"]='changefeed_fast_fail batch_update_to_no_batch changefeed_resume_with_checkpoint_ts' - ["G20"]='tidb_mysql_test ddl_reentrant multi_cdc_cluster multi_changefeed' + ["G20"]='tidb_mysql_test ddl_reentrant multi_cdc_cluster' ["G21"]='bank kill_owner_with_ddl owner_remove_table_error' )