diff --git a/cdc/api/v2/model.go b/cdc/api/v2/model.go index 2e35b8deac3..e9104a3298d 100644 --- a/cdc/api/v2/model.go +++ b/cdc/api/v2/model.go @@ -354,18 +354,20 @@ func (c *ReplicaConfig) toInternalReplicaConfigWithOriginConfig( SASLOAuthGrantType: c.Sink.KafkaConfig.SASLOAuthGrantType, SASLOAuthAudience: c.Sink.KafkaConfig.SASLOAuthAudience, LargeMessageHandle: largeMessageHandle, + OutputRawChangeEvent: c.Sink.KafkaConfig.OutputRawChangeEvent, } } if c.Sink.CloudStorageConfig != nil { res.Sink.CloudStorageConfig = &config.CloudStorageConfig{ - WorkerCount: c.Sink.CloudStorageConfig.WorkerCount, - FlushInterval: c.Sink.CloudStorageConfig.FlushInterval, - FileSize: c.Sink.CloudStorageConfig.FileSize, - FlushConcurrency: c.Sink.CloudStorageConfig.FlushConcurrency, - OutputColumnID: c.Sink.CloudStorageConfig.OutputColumnID, - FileExpirationDays: c.Sink.CloudStorageConfig.FileExpirationDays, - FileCleanupCronSpec: c.Sink.CloudStorageConfig.FileCleanupCronSpec, + WorkerCount: c.Sink.CloudStorageConfig.WorkerCount, + FlushInterval: c.Sink.CloudStorageConfig.FlushInterval, + FileSize: c.Sink.CloudStorageConfig.FileSize, + FlushConcurrency: c.Sink.CloudStorageConfig.FlushConcurrency, + OutputColumnID: c.Sink.CloudStorageConfig.OutputColumnID, + FileExpirationDays: c.Sink.CloudStorageConfig.FileExpirationDays, + FileCleanupCronSpec: c.Sink.CloudStorageConfig.FileCleanupCronSpec, + OutputRawChangeEvent: c.Sink.CloudStorageConfig.OutputRawChangeEvent, } } } @@ -502,18 +504,20 @@ func ToAPIReplicaConfig(c *config.ReplicaConfig) *ReplicaConfig { SASLOAuthGrantType: cloned.Sink.KafkaConfig.SASLOAuthGrantType, SASLOAuthAudience: cloned.Sink.KafkaConfig.SASLOAuthAudience, LargeMessageHandle: largeMessageHandle, + OutputRawChangeEvent: cloned.Sink.KafkaConfig.OutputRawChangeEvent, } } if cloned.Sink.CloudStorageConfig != nil { res.Sink.CloudStorageConfig = &CloudStorageConfig{ - WorkerCount: cloned.Sink.CloudStorageConfig.WorkerCount, - FlushInterval: cloned.Sink.CloudStorageConfig.FlushInterval, - FileSize: cloned.Sink.CloudStorageConfig.FileSize, - FlushConcurrency: cloned.Sink.CloudStorageConfig.FlushConcurrency, - OutputColumnID: cloned.Sink.CloudStorageConfig.OutputColumnID, - FileExpirationDays: cloned.Sink.CloudStorageConfig.FileExpirationDays, - FileCleanupCronSpec: cloned.Sink.CloudStorageConfig.FileCleanupCronSpec, + WorkerCount: cloned.Sink.CloudStorageConfig.WorkerCount, + FlushInterval: cloned.Sink.CloudStorageConfig.FlushInterval, + FileSize: cloned.Sink.CloudStorageConfig.FileSize, + FlushConcurrency: cloned.Sink.CloudStorageConfig.FlushConcurrency, + OutputColumnID: cloned.Sink.CloudStorageConfig.OutputColumnID, + FileExpirationDays: cloned.Sink.CloudStorageConfig.FileExpirationDays, + FileCleanupCronSpec: cloned.Sink.CloudStorageConfig.FileCleanupCronSpec, + OutputRawChangeEvent: cloned.Sink.CloudStorageConfig.OutputRawChangeEvent, } } } @@ -679,18 +683,20 @@ type KafkaConfig struct { SASLOAuthGrantType *string `json:"sasl_oauth_grant_type,omitempty"` SASLOAuthAudience *string `json:"sasl_oauth_audience,omitempty"` - LargeMessageHandle *LargeMessageHandleConfig `json:"large_message_handle,omitempty"` + LargeMessageHandle *LargeMessageHandleConfig `json:"large_message_handle,omitempty"` + OutputRawChangeEvent *bool `json:"output_raw_change_event,omitempty"` } // CloudStorageConfig represents a cloud storage sink configuration type CloudStorageConfig struct { - WorkerCount *int `json:"worker_count,omitempty"` - FlushInterval *string `json:"flush_interval,omitempty"` - FileSize *int `json:"file_size,omitempty"` - FlushConcurrency *int `json:"flush_concurrency,omitempty"` - OutputColumnID *bool `json:"output_column_id,omitempty"` - FileExpirationDays *int `json:"file_expiration_days,omitempty"` - FileCleanupCronSpec *string `json:"file_cleanup_cron_spec,omitempty"` + WorkerCount *int `json:"worker_count,omitempty"` + FlushInterval *string `json:"flush_interval,omitempty"` + FileSize *int `json:"file_size,omitempty"` + FlushConcurrency *int `json:"flush_concurrency,omitempty"` + OutputColumnID *bool `json:"output_column_id,omitempty"` + FileExpirationDays *int `json:"file_expiration_days,omitempty"` + FileCleanupCronSpec *string `json:"file_cleanup_cron_spec,omitempty"` + OutputRawChangeEvent *bool `json:"output_raw_change_event,omitempty"` } // CSVConfig denotes the csv config diff --git a/cdc/entry/mounter.go b/cdc/entry/mounter.go index 4a7d0f8320a..53ac07ea773 100644 --- a/cdc/entry/mounter.go +++ b/cdc/entry/mounter.go @@ -513,6 +513,15 @@ func getDefaultOrZeroValue(col *timodel.ColumnInfo, tz *time.Location) (types.Da if err != nil { return d, d.GetValue(), sizeOfDatum(d), "", errors.Trace(err) } + switch col.GetType() { + case mysql.TypeTimestamp: + t := d.GetMysqlTime() + err = t.ConvertTimeZone(time.UTC, tz) + if err != nil { + return d, d.GetValue(), sizeOfDatum(d), "", errors.Trace(err) + } + d.SetMysqlTime(t) + } } else if !mysql.HasNotNullFlag(col.GetFlag()) { // NOTICE: NotNullCheck need do after OriginDefaultValue check, as when TiDB meet "amend + add column default xxx", // ref: https://github.com/pingcap/ticdc/issues/3929 diff --git a/cdc/entry/mounter_test.go b/cdc/entry/mounter_test.go index c446a1b4e30..cd93ffad1ed 100644 --- a/cdc/entry/mounter_test.go +++ b/cdc/entry/mounter_test.go @@ -875,7 +875,7 @@ func TestGetDefaultZeroValue(t *testing.T) { sc.TimeZone = tz expected, err := types.ParseTimeFromFloatString( sc, - "2020-11-19 12:12:12", colInfo.FieldType.GetType(), colInfo.FieldType.GetDecimal()) + "2020-11-19 20:12:12", colInfo.FieldType.GetType(), colInfo.FieldType.GetDecimal()) require.NoError(t, err) require.Equal(t, expected.String(), val, "mysql.TypeTimestamp + notnull + default") @@ -886,7 +886,7 @@ func TestGetDefaultZeroValue(t *testing.T) { _, val, _, _, _ = getDefaultOrZeroValue(&colInfo, tz) expected, err = types.ParseTimeFromFloatString( sc, - "2020-11-19 12:12:12", colInfo.FieldType.GetType(), colInfo.FieldType.GetDecimal()) + "2020-11-19 20:12:12", colInfo.FieldType.GetType(), colInfo.FieldType.GetDecimal()) require.NoError(t, err) require.Equal(t, expected.String(), val, "mysql.TypeTimestamp + null + default") diff --git a/cdc/kv/client.go b/cdc/kv/client.go index 0541923ed6c..33e64b8aeb8 100644 --- a/cdc/kv/client.go +++ b/cdc/kv/client.go @@ -18,7 +18,6 @@ import ( "fmt" "io" "math/rand" - "strconv" "strings" "sync" "sync/atomic" @@ -39,7 +38,6 @@ import ( "github.com/pingcap/tiflow/pkg/retry" "github.com/pingcap/tiflow/pkg/txnutil" "github.com/pingcap/tiflow/pkg/version" - "github.com/prometheus/client_golang/prometheus" tidbkv "github.com/tikv/client-go/v2/kv" "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/tikv" @@ -75,8 +73,6 @@ const ( resolveLockMinInterval = 10 * time.Second scanRegionsConcurrency = 1024 - - tableMonitorInterval = 2 * time.Second ) // time interval to force kv client to terminate gRPC stream and reconnect @@ -166,7 +162,6 @@ type CDCKVClient interface { ts uint64, lockResolver txnutil.LockResolver, eventCh chan<- model.RegionFeedEvent, - enableTableMonitor bool, ) error // RegionCount returns the number of captured regions. @@ -310,9 +305,8 @@ func (c *CDCClient) EventFeed( ctx context.Context, span regionspan.ComparableSpan, ts uint64, lockResolver txnutil.LockResolver, eventCh chan<- model.RegionFeedEvent, - enableTableMonitor bool, ) error { - s := newEventFeedSession(c, span, lockResolver, ts, eventCh, enableTableMonitor) + s := newEventFeedSession(c, span, lockResolver, ts, eventCh) return s.eventFeed(ctx) } @@ -396,11 +390,6 @@ type eventFeedSession struct { rangeLock *regionspan.RegionRangeLock - enableTableMonitor bool - regionChSizeGauge prometheus.Gauge - errChSizeGauge prometheus.Gauge - rangeChSizeGauge prometheus.Gauge - // storeStreamsCache is used to cache the established gRPC streams to TiKV stores. // Note: The cache is not thread-safe, so it should be accessed in the same goroutine. // For now, it is only accessed in the `requestRegionToStore` goroutine. @@ -421,7 +410,6 @@ func newEventFeedSession( lockResolver txnutil.LockResolver, startTs uint64, eventCh chan<- model.RegionFeedEvent, - enableTableMonitor bool, ) *eventFeedSession { id := allocateRequestID() rangeLock := regionspan.NewRegionRangeLock( @@ -429,23 +417,16 @@ func newEventFeedSession( client.changefeed.Namespace+"."+client.changefeed.ID) return &eventFeedSession{ - client: client, - startTs: startTs, - changefeed: client.changefeed, - tableID: client.tableID, - tableName: client.tableName, - storeStreamsCache: make(map[string]*eventFeedStream), - totalSpan: totalSpan, - eventCh: eventCh, - rangeLock: rangeLock, - lockResolver: lockResolver, - enableTableMonitor: enableTableMonitor, - regionChSizeGauge: clientChannelSize.WithLabelValues(client.changefeed.Namespace, - client.changefeed.ID, strconv.FormatInt(client.tableID, 10), "region"), - errChSizeGauge: clientChannelSize.WithLabelValues(client.changefeed.Namespace, - client.changefeed.ID, strconv.FormatInt(client.tableID, 10), "err"), - rangeChSizeGauge: clientChannelSize.WithLabelValues(client.changefeed.Namespace, - client.changefeed.ID, strconv.FormatInt(client.tableID, 10), "range"), + client: client, + startTs: startTs, + changefeed: client.changefeed, + tableID: client.tableID, + tableName: client.tableName, + storeStreamsCache: make(map[string]*eventFeedStream), + totalSpan: totalSpan, + eventCh: eventCh, + rangeLock: rangeLock, + lockResolver: lockResolver, resolvedTsPool: sync.Pool{ New: func() any { return ®ionStatefulEvent{ @@ -486,7 +467,6 @@ func (s *eventFeedSession) eventFeed(ctx context.Context) error { case <-ctx.Done(): return ctx.Err() case task := <-s.requestRangeCh.Out(): - s.rangeChSizeGauge.Dec() // divideAndSendEventFeedToRegions could be blocked for some time, // since it must wait for the region lock available. In order to // consume region range request from `requestRangeCh` as soon as @@ -508,7 +488,6 @@ func (s *eventFeedSession) eventFeed(ctx context.Context) error { case <-ctx.Done(): return ctx.Err() case errInfo := <-s.errCh.Out(): - s.errChSizeGauge.Dec() if err := s.handleError(ctx, errInfo); err != nil { return err } @@ -518,7 +497,6 @@ func (s *eventFeedSession) eventFeed(ctx context.Context) error { }) s.requestRangeCh.In() <- rangeRequestTask{span: s.totalSpan} - s.rangeChSizeGauge.Inc() log.Info("event feed started", zap.String("namespace", s.changefeed.Namespace), @@ -539,7 +517,6 @@ func (s *eventFeedSession) scheduleDivideRegionAndRequest( task := rangeRequestTask{span: span} select { case s.requestRangeCh.In() <- task: - s.rangeChSizeGauge.Inc() case <-ctx.Done(): } } @@ -553,7 +530,6 @@ func (s *eventFeedSession) scheduleRegionRequest(ctx context.Context, sri single sri.lockedRange = res.LockedRange select { case s.regionCh.In() <- sri: - s.regionChSizeGauge.Inc() case <-ctx.Done(): } case regionspan.LockRangeStatusStale: @@ -615,7 +591,6 @@ func (s *eventFeedSession) onRegionFail(ctx context.Context, errorInfo regionErr zap.Error(errorInfo.err)) select { case s.errCh.In() <- errorInfo: - s.errChSizeGauge.Inc() case <-ctx.Done(): } } @@ -791,7 +766,6 @@ func (s *eventFeedSession) dispatchRequest(ctx context.Context) error { case <-ctx.Done(): return errors.Trace(ctx.Err()) case sri = <-s.regionCh.Out(): - s.regionChSizeGauge.Dec() } // Send a resolved ts to event channel first, for two reasons: @@ -1040,11 +1014,6 @@ func (s *eventFeedSession) receiveFromStream( metricSendEventBatchResolvedSize := batchResolvedEventSize. WithLabelValues(s.changefeed.Namespace, s.changefeed.ID) - metricReceiveBusyRatio := workerBusyRatio.WithLabelValues( - s.changefeed.Namespace, s.changefeed.ID, strconv.FormatInt(s.tableID, 10), stream.addr, "event-receiver") - metricProcessBusyRatio := workerBusyRatio.WithLabelValues( - s.changefeed.Namespace, s.changefeed.ID, strconv.FormatInt(s.tableID, 10), stream.addr, "event-processor") - // always create a new region worker, because `receiveFromStream` is ensured // to call exactly once from outer code logic worker := newRegionWorker(parentCtx, stream, s) @@ -1063,7 +1032,7 @@ func (s *eventFeedSession) receiveFromStream( eg, ctx := errgroup.WithContext(ctx) eg.Go(func() error { - err := handleExit(worker.run(s.enableTableMonitor)) + err := handleExit(worker.run()) if err != nil { log.Error("region worker exited with error", zap.String("namespace", s.changefeed.Namespace), @@ -1079,32 +1048,10 @@ func (s *eventFeedSession) receiveFromStream( }) receiveEvents := func() error { - var receiveTime time.Duration - var processTime time.Duration - startToWork := time.Now() - maxCommitTs := model.Ts(0) for { - startToReceive := time.Now() cevent, err := stream.client.Recv() - if s.enableTableMonitor { - receiveTime += time.Since(startToReceive) - if time.Since(startToWork) >= tableMonitorInterval { - now := time.Now() - // Receive busyRatio indicates the blocking time (receive and decode grpc msg) of the worker. - busyRatio := receiveTime.Seconds() / now.Sub(startToWork).Seconds() * 100 - metricReceiveBusyRatio.Set(busyRatio) - receiveTime = 0 - // Process busyRatio indicates the working time (dispatch to region worker) of the worker. - busyRatio = processTime.Seconds() / now.Sub(startToWork).Seconds() * 100 - metricProcessBusyRatio.Set(busyRatio) - processTime = 0 - - startToWork = now - } - } - failpoint.Inject("kvClientRegionReentrantError", func(op failpoint.Value) { if op.(string) == "error" { _ = worker.sendEvents(ctx, []*regionStatefulEvent{nil}) @@ -1163,7 +1110,6 @@ func (s *eventFeedSession) receiveFromStream( return nil } - startToProcess := time.Now() size := cevent.Size() if size > warnRecvMsgSizeThreshold { regionCount := 0 @@ -1208,7 +1154,6 @@ func (s *eventFeedSession) receiveFromStream( tsStat.commitTs.Store(maxCommitTs) } } - processTime += time.Since(startToProcess) } } eg.Go(func() error { diff --git a/cdc/kv/client_bench_test.go b/cdc/kv/client_bench_test.go index c517e71afc9..4c00a77064c 100644 --- a/cdc/kv/client_bench_test.go +++ b/cdc/kv/client_bench_test.go @@ -202,7 +202,7 @@ func prepareBenchMultiStore(b *testing.B, storeNum, regionNum int) ( go func() { err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) if errors.Cause(err) != context.Canceled { b.Error(err) } @@ -296,7 +296,7 @@ func prepareBench(b *testing.B, regionNum int) ( go func() { err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("z")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) if errors.Cause(err) != context.Canceled { b.Error(err) } diff --git a/cdc/kv/client_test.go b/cdc/kv/client_test.go index 664b1dfcec8..89b2d53d629 100644 --- a/cdc/kv/client_test.go +++ b/cdc/kv/client_test.go @@ -330,7 +330,7 @@ func TestConnectOfflineTiKV(t *testing.T) { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, - 1, lockResolver, eventCh, false) + 1, lockResolver, eventCh) require.Equal(t, context.Canceled, errors.Cause(err)) }() @@ -428,7 +428,7 @@ func TestRecvLargeMessageSize(t *testing.T) { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, - 1, lockResolver, eventCh, false) + 1, lockResolver, eventCh) require.Equal(t, context.Canceled, errors.Cause(err)) }() @@ -528,7 +528,7 @@ func TestHandleError(t *testing.T) { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("d")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) require.Equal(t, context.Canceled, errors.Cause(err)) }() @@ -688,7 +688,7 @@ func TestCompatibilityWithSameConn(t *testing.T) { defer wg2.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) require.True(t, cerror.ErrVersionIncompatible.Equal(err)) }() @@ -756,7 +756,7 @@ func TestClusterIDMismatch(t *testing.T) { defer wg2.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) require.True(t, cerror.ErrClusterIDMismatch.Equal(err)) }() @@ -823,7 +823,7 @@ func testHandleFeedEvent(t *testing.T) { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) require.Equal(t, context.Canceled, errors.Cause(err)) }() @@ -1284,7 +1284,7 @@ func TestStreamSendWithError(t *testing.T) { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("c")}, - 100, lockerResolver, eventCh, false) + 100, lockerResolver, eventCh) require.Equal(t, context.Canceled, errors.Cause(err)) }() @@ -1402,7 +1402,7 @@ func testStreamRecvWithError(t *testing.T, failpointStr string) { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) require.Equal(t, context.Canceled, errors.Cause(err)) }() @@ -1537,7 +1537,7 @@ func TestStreamRecvWithErrorAndResolvedGoBack(t *testing.T) { defer close(eventCh) err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) require.Equal(t, context.Canceled, errors.Cause(err)) }() @@ -1746,7 +1746,7 @@ func TestIncompatibleTiKV(t *testing.T) { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) require.Equal(t, context.Canceled, errors.Cause(err)) }() @@ -1823,7 +1823,7 @@ func TestNoPendingRegionError(t *testing.T) { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) require.Equal(t, context.Canceled, errors.Cause(err)) }() @@ -1901,7 +1901,7 @@ func TestDropStaleRequest(t *testing.T) { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) require.Equal(t, context.Canceled, errors.Cause(err)) }() @@ -2015,7 +2015,7 @@ func TestResolveLock(t *testing.T) { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) require.Equal(t, context.Canceled, errors.Cause(err)) }() @@ -2123,7 +2123,7 @@ func testEventCommitTsFallback(t *testing.T, events []*cdcpb.ChangeDataEvent) { defer clientWg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) require.Equal(t, errUnreachable, err) }() @@ -2250,7 +2250,7 @@ func testEventAfterFeedStop(t *testing.T) { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) require.Equal(t, context.Canceled, errors.Cause(err)) }() @@ -2437,7 +2437,7 @@ func TestOutOfRegionRangeEvent(t *testing.T) { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) require.Equal(t, context.Canceled, errors.Cause(err)) }() @@ -2655,7 +2655,7 @@ func TestResolveLockNoCandidate(t *testing.T) { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) require.Equal(t, context.Canceled, errors.Cause(err)) }() @@ -2751,7 +2751,7 @@ func TestFailRegionReentrant(t *testing.T) { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) require.Equal(t, context.Canceled, errors.Cause(err)) }() @@ -2834,7 +2834,7 @@ func TestClientV1UnlockRangeReentrant(t *testing.T) { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("c")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) require.Equal(t, context.Canceled, errors.Cause(err)) }() @@ -2902,7 +2902,7 @@ func testClientErrNoPendingRegion(t *testing.T) { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("c")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) require.Equal(t, context.Canceled, errors.Cause(err)) }() @@ -2980,7 +2980,7 @@ func testKVClientForceReconnect(t *testing.T) { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("c")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) require.Equal(t, context.Canceled, errors.Cause(err)) }() @@ -3131,7 +3131,7 @@ func TestConcurrentProcessRangeRequest(t *testing.T) { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("z")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) require.Equal(t, context.Canceled, errors.Cause(err)) }() @@ -3248,7 +3248,7 @@ func TestEvTimeUpdate(t *testing.T) { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) require.Equal(t, context.Canceled, errors.Cause(err)) }() @@ -3374,7 +3374,7 @@ func TestRegionWorkerExitWhenIsIdle(t *testing.T) { defer wg.Done() err := cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("b")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) require.Equal(t, context.Canceled, errors.Cause(err)) }() @@ -3468,7 +3468,7 @@ func TestPrewriteNotMatchError(t *testing.T) { defer wg.Done() err = cdcClient.EventFeed(ctx, regionspan.ComparableSpan{Start: []byte("a"), End: []byte("c")}, - 100, lockResolver, eventCh, false) + 100, lockResolver, eventCh) require.Equal(t, context.Canceled, errors.Cause(err)) }() @@ -3546,6 +3546,5 @@ func createFakeEventFeedSession() *eventFeedSession { nil, /*lockResolver*/ 100, /*startTs*/ nil, /*eventCh*/ - false, ) } diff --git a/cdc/kv/metrics.go b/cdc/kv/metrics.go index 31142175274..4f063ed470d 100644 --- a/cdc/kv/metrics.go +++ b/cdc/kv/metrics.go @@ -71,7 +71,7 @@ var ( Subsystem: "kvclient", Name: "channel_size", Help: "size of each channel in kv client", - }, []string{"namespace", "changefeed", "table", "type"}) + }, []string{"channel"}) clientRegionTokenSize = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "ticdc", @@ -110,20 +110,6 @@ var ( Help: "region events batch size", Buckets: prometheus.ExponentialBuckets(1, 2, 20), }) - workerBusyRatio = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "kvclient", - Name: "region_worker_busy_ratio", - Help: "Busy ratio (X ms in 1s) for region worker.", - }, []string{"namespace", "changefeed", "table", "store", "type"}) - workerChannelSize = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "kvclient", - Name: "region_worker_channel_size", - Help: "size of each channel in region worker", - }, []string{"namespace", "changefeed", "table", "store", "type"}) ) // InitMetrics registers all metrics in the kv package @@ -140,8 +126,6 @@ func InitMetrics(registry *prometheus.Registry) { registry.MustRegister(batchResolvedEventSize) registry.MustRegister(grpcPoolStreamGauge) registry.MustRegister(regionEventsBatchSize) - registry.MustRegister(workerBusyRatio) - registry.MustRegister(workerChannelSize) // Register client metrics to registry. registry.MustRegister(grpcMetrics) diff --git a/cdc/kv/region_worker.go b/cdc/kv/region_worker.go index 150e82975eb..5ab91216855 100644 --- a/cdc/kv/region_worker.go +++ b/cdc/kv/region_worker.go @@ -18,7 +18,6 @@ import ( "encoding/hex" "reflect" "runtime" - "strconv" "sync" "sync/atomic" "time" @@ -75,9 +74,6 @@ type regionWorkerMetrics struct { metricSendEventResolvedCounter prometheus.Counter metricSendEventCommitCounter prometheus.Counter metricSendEventCommittedCounter prometheus.Counter - - metricWorkerBusyRatio prometheus.Gauge - metricWorkerChannelSize prometheus.Gauge } /* @@ -117,7 +113,7 @@ type regionWorker struct { inputPendingEvents int32 } -func newRegionWorkerMetrics(changefeedID model.ChangeFeedID, tableID string, storeAddr string) *regionWorkerMetrics { +func newRegionWorkerMetrics(changefeedID model.ChangeFeedID) *regionWorkerMetrics { metrics := ®ionWorkerMetrics{} metrics.metricReceivedEventSize = eventSize.WithLabelValues("received") metrics.metricDroppedEventSize = eventSize.WithLabelValues("dropped") @@ -140,11 +136,6 @@ func newRegionWorkerMetrics(changefeedID model.ChangeFeedID, tableID string, sto metrics.metricSendEventCommittedCounter = sendEventCounter. WithLabelValues("committed", changefeedID.Namespace, changefeedID.ID) - metrics.metricWorkerBusyRatio = workerBusyRatio.WithLabelValues( - changefeedID.Namespace, changefeedID.ID, tableID, storeAddr, "event-handler") - metrics.metricWorkerChannelSize = workerChannelSize.WithLabelValues( - changefeedID.Namespace, changefeedID.ID, tableID, storeAddr, "input") - return metrics } @@ -164,7 +155,7 @@ func newRegionWorker( rtsManager: newRegionTsManager(), rtsUpdateCh: make(chan *rtsUpdateEvent, 1024), concurrency: s.client.config.KVClient.WorkerConcurrent, - metrics: newRegionWorkerMetrics(s.changefeed, strconv.FormatInt(s.tableID, 10), stream.addr), + metrics: newRegionWorkerMetrics(s.changefeed), inputPendingEvents: 0, } } @@ -458,7 +449,7 @@ func (w *regionWorker) onHandleExit(err error) { } } -func (w *regionWorker) eventHandler(ctx context.Context, enableTableMonitor bool) error { +func (w *regionWorker) eventHandler(ctx context.Context) error { exitFn := func() error { log.Info("region worker closed by error", zap.String("namespace", w.session.client.changefeed.Namespace), @@ -468,11 +459,6 @@ func (w *regionWorker) eventHandler(ctx context.Context, enableTableMonitor bool return cerror.ErrRegionWorkerExit.GenWithStackByArgs() } - metricsTicker := time.NewTicker(tableMonitorInterval) - defer metricsTicker.Stop() - var processTime time.Duration - startToWork := time.Now() - highWatermarkMet := false for { select { @@ -480,17 +466,6 @@ func (w *regionWorker) eventHandler(ctx context.Context, enableTableMonitor bool return errors.Trace(ctx.Err()) case err := <-w.errorCh: return errors.Trace(err) - case <-metricsTicker.C: - if enableTableMonitor { - w.metrics.metricWorkerChannelSize.Set(float64(len(w.inputCh))) - - now := time.Now() - // busyRatio indicates the actual working time of the worker. - busyRatio := processTime.Seconds() / now.Sub(startToWork).Seconds() * 100 - w.metrics.metricWorkerBusyRatio.Set(busyRatio) - startToWork = now - processTime = 0 - } case events, ok := <-w.inputCh: if !ok { return exitFn() @@ -507,8 +482,6 @@ func (w *regionWorker) eventHandler(ctx context.Context, enableTableMonitor bool } regionEventsBatchSize.Observe(float64(len(events))) - - start := time.Now() inputPending := atomic.LoadInt32(&w.inputPendingEvents) if highWatermarkMet { highWatermarkMet = int(inputPending) >= regionWorkerLowWatermark @@ -569,7 +542,6 @@ func (w *regionWorker) eventHandler(ctx context.Context, enableTableMonitor bool w.session.resolvedTsPool.Put(ev) } } - processTime += time.Since(start) } } } @@ -626,7 +598,7 @@ func (w *regionWorker) cancelStream(delay time.Duration) { time.Sleep(delay) } -func (w *regionWorker) run(enableTableMonitor bool) error { +func (w *regionWorker) run() error { defer func() { for _, h := range w.handles { h.Unregister() @@ -651,7 +623,7 @@ func (w *regionWorker) run(enableTableMonitor bool) error { return handleError(w.checkErrorReconnect(w.resolveLock(ctx))) }) wg.Go(func() error { - return handleError(w.eventHandler(ctx, enableTableMonitor)) + return handleError(w.eventHandler(ctx)) }) _ = handleError(w.collectWorkpoolError(ctx)) _ = wg.Wait() diff --git a/cdc/model/kv.go b/cdc/model/kv.go index 0c5fe65d5c7..1b0977b59bb 100644 --- a/cdc/model/kv.go +++ b/cdc/model/kv.go @@ -16,6 +16,7 @@ package model import ( + "errors" "fmt" "github.com/pingcap/tiflow/pkg/regionspan" @@ -110,3 +111,20 @@ func (v *RawKVEntry) String() string { func (v *RawKVEntry) ApproximateDataSize() int64 { return int64(len(v.Key) + len(v.Value) + len(v.OldValue)) } + +// ShouldSplitKVEntry checks whether the raw kv entry should be splitted. +type ShouldSplitKVEntry func(raw *RawKVEntry) bool + +// SplitUpdateKVEntry splits the raw kv entry into a delete entry and an insert entry. +func SplitUpdateKVEntry(raw *RawKVEntry) (*RawKVEntry, *RawKVEntry, error) { + if raw == nil { + return nil, nil, errors.New("nil event cannot be split") + } + deleteKVEntry := *raw + deleteKVEntry.Value = nil + + insertKVEntry := *raw + insertKVEntry.OldValue = nil + + return &deleteKVEntry, &insertKVEntry, nil +} diff --git a/cdc/model/sink.go b/cdc/model/sink.go index f479e68a565..67ca84578a7 100644 --- a/cdc/model/sink.go +++ b/cdc/model/sink.go @@ -279,7 +279,7 @@ func (r *RedoLog) GetCommitTs() Ts { } // TrySplitAndSortUpdateEvent redo log do nothing -func (r *RedoLog) TrySplitAndSortUpdateEvent(sinkScheme string) error { +func (r *RedoLog) TrySplitAndSortUpdateEvent(_ string, _ bool) error { return nil } @@ -377,7 +377,7 @@ func (r *RowChangedEvent) GetCommitTs() uint64 { } // TrySplitAndSortUpdateEvent do nothing -func (r *RowChangedEvent) TrySplitAndSortUpdateEvent(sinkScheme string) error { +func (r *RowChangedEvent) TrySplitAndSortUpdateEvent(_ string, _ bool) error { return nil } @@ -794,11 +794,19 @@ func (t *SingleTableTxn) GetCommitTs() uint64 { } // TrySplitAndSortUpdateEvent split update events if unique key is updated -func (t *SingleTableTxn) TrySplitAndSortUpdateEvent(sinkScheme string) error { - if !t.shouldSplitUpdateEvent(sinkScheme) { +func (t *SingleTableTxn) TrySplitAndSortUpdateEvent(scheme string, outputRawChangeEvent bool) error { + if sink.IsMySQLCompatibleScheme(scheme) || outputRawChangeEvent { + // For MySQL Sink, all update events will be split into insert and delete at the puller side + // according to whether the changefeed is in safemode. We don't split update event here(in sink) + // since there may be OOM issues. For more information, ref https://github.com/tikv/tikv/issues/17062. + // + // For the Kafka and Storage sink, the outputRawChangeEvent parameter is introduced to control + // split behavior. TiCDC only output original change event if outputRawChangeEvent is true. return nil } + // Try to split update events for the Kafka and Storage sink if outputRawChangeEvent is false. + // Note it is only for backward compatibility, and we should remove this logic in the future. newRows, err := trySplitAndSortUpdateEvent(t.Rows) if err != nil { return errors.Trace(err) @@ -807,21 +815,6 @@ func (t *SingleTableTxn) TrySplitAndSortUpdateEvent(sinkScheme string) error { return nil } -// Whether split a single update event into delete and insert events? -// -// For the MySQL Sink, we don't split any update event. -// This may cause error like "duplicate entry" when sink to the downstream. -// This kind of error will cause the changefeed to restart, -// and then the related update rows will be splitted to insert and delete at puller side. -// -// 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 { - return !sink.IsMySQLCompatibleScheme(sinkScheme) -} - // trySplitAndSortUpdateEvent try to split update events if unique key is updated // returns true if some updated events is split func trySplitAndSortUpdateEvent( @@ -831,8 +824,7 @@ func trySplitAndSortUpdateEvent( split := false for _, e := range events { if e == nil { - log.Warn("skip emit nil event", - zap.Any("event", e)) + log.Warn("skip emit nil event", zap.Any("event", e)) continue } @@ -842,8 +834,7 @@ func trySplitAndSortUpdateEvent( // 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 { - log.Warn("skip emit empty row event", - zap.Any("event", e)) + log.Warn("skip emit empty row event", zap.Any("event", e)) continue } @@ -869,7 +860,7 @@ func trySplitAndSortUpdateEvent( // ShouldSplitUpdateEvent determines if the split event is needed to align the old format based on // whether the handle key column or unique key has been modified. -// If is modified, we need to use splitUpdateEvent to split the update event into a delete and an insert event. +// If is modified, we need to use splitUpdateEvent to split the update event into a delete and an insert event. func ShouldSplitUpdateEvent(updateEvent *RowChangedEvent) bool { // nil event will never be split. if updateEvent == nil { @@ -912,6 +903,11 @@ func SplitUpdateEvent( // NOTICE: clean up pre cols for insert event. insertEvent.PreColumns = nil + log.Debug("split update event", zap.Uint64("startTs", updateEvent.StartTs), + zap.Uint64("commitTs", updateEvent.CommitTs), + zap.Any("preCols", updateEvent.PreColumns), + zap.Any("cols", updateEvent.Columns)) + return &deleteEvent, &insertEvent, nil } diff --git a/cdc/model/sink_test.go b/cdc/model/sink_test.go index 4041c673958..690f3ae5497 100644 --- a/cdc/model/sink_test.go +++ b/cdc/model/sink_test.go @@ -534,6 +534,7 @@ func TestTrySplitAndSortUpdateEvent(t *testing.T) { events := []*RowChangedEvent{ { + TableInfo: &TableInfo{}, CommitTs: 1, Columns: columns, PreColumns: preColumns, @@ -573,6 +574,7 @@ func TestTrySplitAndSortUpdateEvent(t *testing.T) { events = []*RowChangedEvent{ { + TableInfo: &TableInfo{}, CommitTs: 1, Columns: columns, PreColumns: preColumns, @@ -613,6 +615,7 @@ func TestTrySplitAndSortUpdateEvent(t *testing.T) { events = []*RowChangedEvent{ { + TableInfo: &TableInfo{}, CommitTs: 1, Columns: columns, PreColumns: preColumns, @@ -624,6 +627,12 @@ func TestTrySplitAndSortUpdateEvent(t *testing.T) { } var ukUpdatedEvent = &RowChangedEvent{ + TableInfo: &TableInfo{ + TableName: TableName{ + Schema: "test", + Table: "t1", + }, + }, PreColumns: []*Column{ { Name: "col1", @@ -656,21 +665,32 @@ func TestTrySplitAndSortUpdateEventOne(t *testing.T) { Rows: []*RowChangedEvent{ukUpdatedEvent}, } - err := txn.TrySplitAndSortUpdateEvent(sink.KafkaScheme) + outputRawChangeEvent := true + notOutputRawChangeEvent := false + err := txn.TrySplitAndSortUpdateEvent(sink.KafkaScheme, outputRawChangeEvent) + require.NoError(t, err) + require.Len(t, txn.Rows, 1) + err = txn.TrySplitAndSortUpdateEvent(sink.KafkaScheme, notOutputRawChangeEvent) require.NoError(t, err) require.Len(t, txn.Rows, 2) txn = &SingleTableTxn{ Rows: []*RowChangedEvent{ukUpdatedEvent}, } - err = txn.TrySplitAndSortUpdateEvent(sink.MySQLScheme) + err = txn.TrySplitAndSortUpdateEvent(sink.MySQLScheme, outputRawChangeEvent) + require.NoError(t, err) + require.Len(t, txn.Rows, 1) + err = txn.TrySplitAndSortUpdateEvent(sink.MySQLScheme, notOutputRawChangeEvent) require.NoError(t, err) require.Len(t, txn.Rows, 1) txn2 := &SingleTableTxn{ Rows: []*RowChangedEvent{ukUpdatedEvent, ukUpdatedEvent}, } - err = txn.TrySplitAndSortUpdateEvent(sink.MySQLScheme) + err = txn.TrySplitAndSortUpdateEvent(sink.MySQLScheme, outputRawChangeEvent) + require.NoError(t, err) + require.Len(t, txn2.Rows, 2) + err = txn.TrySplitAndSortUpdateEvent(sink.MySQLScheme, notOutputRawChangeEvent) require.NoError(t, err) require.Len(t, txn2.Rows, 2) } diff --git a/cdc/owner/ddl_manager.go b/cdc/owner/ddl_manager.go index 43c66686c38..13fd237b3c3 100644 --- a/cdc/owner/ddl_manager.go +++ b/cdc/owner/ddl_manager.go @@ -224,24 +224,7 @@ func (m *ddlManager) tick( } for _, event := range events { - // TODO: find a better place to do this check - // check if the ddl event is belong to an ineligible table. - // If so, we should ignore it. - if !filter.IsSchemaDDL(event.Type) { - ignore, err := m.schema. - IsIneligibleTable(ctx, event.TableInfo.TableName.TableID, event.CommitTs) - if err != nil { - return nil, nil, errors.Trace(err) - } - if ignore { - log.Warn("ignore the DDL event of ineligible table", - zap.String("changefeed", m.changfeedID.ID), zap.Any("ddl", event)) - continue - } - } - tableName := event.TableInfo.TableName - // Add all valid DDL events to the pendingDDLs. m.pendingDDLs[tableName] = append(m.pendingDDLs[tableName], event) } diff --git a/cdc/processor/pipeline/puller.go b/cdc/processor/pipeline/puller.go index 5e5b63d9e44..188b0167e97 100644 --- a/cdc/processor/pipeline/puller.go +++ b/cdc/processor/pipeline/puller.go @@ -85,7 +85,6 @@ func (n *pullerNode) startWithSorterNode(ctx pipeline.NodeContext, n.tableID, n.tableName, filterLoop, - false, ) n.wg.Go(func() error { ctx.Throw(errors.Trace(n.plr.Run(ctxC))) diff --git a/cdc/processor/processor.go b/cdc/processor/processor.go index 4ae710e7a7e..865f2d72485 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -219,11 +219,11 @@ func (p *processor) AddTable( } if p.pullBasedSinking { - p.sinkManager.AddTable(tableID, startTs, p.changefeed.Info.TargetTs) + table := p.sinkManager.AddTable(tableID, startTs, p.changefeed.Info.TargetTs) if p.redoDMLMgr.Enabled() { p.redoDMLMgr.AddTable(tableID, startTs) } - p.sourceManager.AddTable(ctx.(cdcContext.Context), tableID, p.getTableName(ctx, tableID), startTs) + p.sourceManager.AddTable(ctx.(cdcContext.Context), tableID, p.getTableName(ctx, tableID), startTs, table.GetReplicaTs) } else { table, err := p.createTablePipeline( ctx.(cdcContext.Context), tableID, &model.TableReplicaInfo{StartTs: startTs}) @@ -232,7 +232,6 @@ func (p *processor) AddTable( } p.tables[tableID] = table } - return true, nil } @@ -570,18 +569,6 @@ func isProcessorIgnorableError(err error) bool { return false } -// needPullerSafeModeAtStart returns true if the scheme is mysql compatible. -// pullerSafeMode means to split all update kv entries whose commitTS -// is older then the start time of this changefeed. -func needPullerSafeModeAtStart(sinkURIStr string) (bool, error) { - sinkURI, err := url.Parse(sinkURIStr) - if err != nil { - return false, cerror.WrapError(cerror.ErrSinkURIInvalid, err) - } - scheme := sink.GetScheme(sinkURI) - return sink.IsMySQLCompatibleScheme(scheme), nil -} - // Tick implements the `orchestrator.State` interface // the `state` parameter is sent by the etcd worker, the `state` must be a snapshot of KVs in etcd // The main logic of processor is in this function, including the calculation of many kinds of ts, @@ -757,6 +744,16 @@ func (p *processor) createTaskPosition() (skipThisTick bool) { return true } +// isMysqlCompatibleBackend returns true if the sinkURIStr is mysql compatible. +func isMysqlCompatibleBackend(sinkURIStr string) (bool, error) { + sinkURI, err := url.Parse(sinkURIStr) + if err != nil { + return false, cerror.WrapError(cerror.ErrSinkURIInvalid, err) + } + scheme := sink.GetScheme(sinkURI) + return sink.IsMySQLCompatibleScheme(scheme), nil +} + // lazyInitImpl create Filter, SchemaStorage, Mounter instances at the first tick. func (p *processor) lazyInitImpl(ctx cdcContext.Context) error { if p.initialized { @@ -847,16 +844,16 @@ func (p *processor) lazyInitImpl(ctx cdcContext.Context) error { zap.Duration("duration", time.Since(start))) return errors.Trace(err) } - pullerSafeModeAtStart, err := needPullerSafeModeAtStart(p.changefeed.Info.SinkURI) + isMysqlBackend, err := isMysqlCompatibleBackend(p.changefeed.Info.SinkURI) if err != nil { return errors.Trace(err) } p.sourceManager = sourcemanager.New(p.changefeedID, p.upstream, p.mg, - sortEngine, p.errCh, p.changefeed.Info.Config.BDRMode, p.changefeed.Info.Config.EnableTableMonitor, pullerSafeModeAtStart) + sortEngine, p.errCh, p.changefeed.Info.Config.BDRMode, isMysqlBackend) p.sinkManager, err = sinkmanager.New(stdCtx, p.changefeedID, p.changefeed.Info, p.upstream, p.schemaStorage, p.redoDMLMgr, p.sourceManager, - p.errCh, p.warnCh, p.metricsTableSinkTotalRows, p.metricsTableSinkFlushLagDuration) + p.errCh, p.warnCh, isMysqlBackend, p.metricsTableSinkTotalRows, p.metricsTableSinkFlushLagDuration) if err != nil { log.Info("Processor creates sink manager fail", zap.String("namespace", p.changefeedID.Namespace), diff --git a/cdc/processor/sinkmanager/manager.go b/cdc/processor/sinkmanager/manager.go index 5801343903a..85d8099e23e 100644 --- a/cdc/processor/sinkmanager/manager.go +++ b/cdc/processor/sinkmanager/manager.go @@ -128,6 +128,9 @@ type SinkManager struct { // wg is used to wait for all workers to exit. wg sync.WaitGroup + // isMysqlBackend indicates whether the backend is MySQL compatible. + isMysqlBackend bool + // Metric for table sink. metricsTableSinkTotalRows prometheus.Counter @@ -145,6 +148,7 @@ func New( sourceManager *sourcemanager.SourceManager, errChan chan error, warnChan chan error, + isMysqlBackend bool, metricsTableSinkTotalRows prometheus.Counter, metricsTableSinkFlushLagDuration prometheus.Observer, ) (*SinkManager, error) { @@ -160,6 +164,7 @@ func New( sinkTaskChan: make(chan *sinkTask), sinkWorkerAvailable: make(chan struct{}, 1), sinkRetry: retry.NewInfiniteErrorRetry(), + isMysqlBackend: isMysqlBackend, metricsTableSinkTotalRows: metricsTableSinkTotalRows, @@ -312,6 +317,11 @@ func (m *SinkManager) run(ctx context.Context, warnings ...chan<- error) (err er if cerror.IsDupEntryError(err) { return errors.Trace(err) } + + if m.isMysqlBackend { + // For MySQL backend, we should restart sink. Let owner to handle the error. + return errors.Trace(err) + } } // If the error is retryable, we should retry to re-establish the internal resources. @@ -848,7 +858,7 @@ func (m *SinkManager) UpdateBarrierTs(globalBarrierTs model.Ts, tableBarrier map } // AddTable adds a table(TableSink) to the sink manager. -func (m *SinkManager) AddTable(tableID model.TableID, startTs model.Ts, targetTs model.Ts) { +func (m *SinkManager) AddTable(tableID model.TableID, startTs model.Ts, targetTs model.Ts) *tableSinkWrapper { sinkWrapper := newTableSinkWrapper( m.changefeedID, tableID, @@ -876,7 +886,6 @@ func (m *SinkManager) AddTable(tableID model.TableID, startTs model.Ts, targetTs zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), zap.Int64("tableID", tableID)) - return } m.sinkMemQuota.AddTable(tableID) m.redoMemQuota.AddTable(tableID) @@ -886,6 +895,7 @@ func (m *SinkManager) AddTable(tableID model.TableID, startTs model.Ts, targetTs zap.Int64("tableID", tableID), zap.Uint64("startTs", startTs), zap.Uint64("version", sinkWrapper.version)) + return sinkWrapper } // StartTable sets the table(TableSink) state to replicating. diff --git a/cdc/processor/sinkmanager/manager_test.go b/cdc/processor/sinkmanager/manager_test.go index bdd75798d06..b6d98f68da4 100644 --- a/cdc/processor/sinkmanager/manager_test.go +++ b/cdc/processor/sinkmanager/manager_test.go @@ -57,12 +57,13 @@ func createManagerWithMemEngine( ) (*SinkManager, engine.SortEngine) { sortEngine := memory.New(context.Background()) up := upstream.NewUpstream4Test(&mockPD{}) - sm := sourcemanager.New(changefeedID, up, &entry.MockMountGroup{}, sortEngine, errChan, false, false, false) + sm := sourcemanager.New(changefeedID, up, &entry.MockMountGroup{}, sortEngine, errChan, false, false) manager, err := New( ctx, changefeedID, changefeedInfo, up, &entry.MockSchemaStorage{Resolved: math.MaxUint64}, nil, sm, errChan, errChan, + false, prometheus.NewCounter(prometheus.CounterOpts{}), prometheus.NewHistogram(prometheus.HistogramOpts{})) require.NoError(t, err) @@ -166,7 +167,7 @@ func TestAddTable(t *testing.T) { require.Equal(t, 0, manager.sinkProgressHeap.len(), "Not started table shout not in progress heap") err := manager.StartTable(tableID, 1) require.NoError(t, err) - require.Equal(t, uint64(0x7ffffffffffbffff), tableSink.(*tableSinkWrapper).replicateTs) + require.Equal(t, uint64(0x7ffffffffffbffff), tableSink.(*tableSinkWrapper).replicateTs.Load()) progress := manager.sinkProgressHeap.pop() require.Equal(t, tableID, progress.tableID) diff --git a/cdc/processor/sinkmanager/redo_log_worker.go b/cdc/processor/sinkmanager/redo_log_worker.go index 2e7b566cd69..a8e4d642e85 100644 --- a/cdc/processor/sinkmanager/redo_log_worker.go +++ b/cdc/processor/sinkmanager/redo_log_worker.go @@ -276,8 +276,8 @@ func (w *redoWorker) handleTask(ctx context.Context, task *redoTask) (finalErr e var x []*model.RowChangedEvent var size uint64 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 + // For all events, we add table replicate ts, so mysql sink can determine safe-mode. + e.Row.ReplicatingTs = task.tableSink.replicateTs.Load() x, size = handleRowChangedEvents(w.changefeedID, task.tableID, e) usedMemSize += size rows = append(rows, x...) diff --git a/cdc/processor/sinkmanager/table_sink_worker.go b/cdc/processor/sinkmanager/table_sink_worker.go index 4a55b92983f..b11e4347d73 100644 --- a/cdc/processor/sinkmanager/table_sink_worker.go +++ b/cdc/processor/sinkmanager/table_sink_worker.go @@ -363,7 +363,7 @@ func (w *sinkWorker) handleTask(ctx context.Context, task *sinkTask) (finalErr e // NOTICE: The event can be filtered by the event filter. 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 + e.Row.ReplicatingTs = task.tableSink.GetReplicaTs() x, size := handleRowChangedEvents(w.changefeedID, task.tableID, e) events = append(events, x...) allEventSize += size diff --git a/cdc/processor/sinkmanager/table_sink_worker_test.go b/cdc/processor/sinkmanager/table_sink_worker_test.go index 33f7ffc8817..bf72629c302 100644 --- a/cdc/processor/sinkmanager/table_sink_worker_test.go +++ b/cdc/processor/sinkmanager/table_sink_worker_test.go @@ -39,7 +39,7 @@ func createWorker( ) (*sinkWorker, engine.SortEngine) { sortEngine := memory.New(context.Background()) sm := sourcemanager.New(changefeedID, upstream.NewUpstream4Test(&mockPD{}), - &entry.MockMountGroup{}, sortEngine, make(chan error, 1), false, false, false) + &entry.MockMountGroup{}, sortEngine, make(chan error, 1), false, false) // To avoid refund or release panics. quota := memquota.NewMemQuota(changefeedID, memQuota+1024*1024*1024, "") diff --git a/cdc/processor/sinkmanager/table_sink_wrapper.go b/cdc/processor/sinkmanager/table_sink_wrapper.go index 6d253413cc7..839b661eec7 100644 --- a/cdc/processor/sinkmanager/table_sink_wrapper.go +++ b/cdc/processor/sinkmanager/table_sink_wrapper.go @@ -15,6 +15,7 @@ package sinkmanager import ( "context" + "math" "sort" "sync" "sync/atomic" @@ -76,7 +77,7 @@ type tableSinkWrapper struct { receivedSorterResolvedTs atomic.Uint64 // replicateTs is the ts that the table sink has started to replicate. - replicateTs model.Ts + replicateTs atomic.Uint64 genReplicateTs func(ctx context.Context) (model.Ts, error) // lastCleanTime indicates the last time the table has been cleaned. @@ -89,6 +90,11 @@ type tableSinkWrapper struct { rangeEventCountsMu sync.Mutex } +// GetReplicaTs returns the replicate ts of the table sink. +func (t *tableSinkWrapper) GetReplicaTs() model.Ts { + return t.replicateTs.Load() +} + type rangeEventCount struct { // firstPos and lastPos are used to merge many rangeEventCount into one. firstPos engine.Position @@ -131,31 +137,34 @@ func newTableSinkWrapper( res.receivedSorterResolvedTs.Store(startTs) res.barrierTs.Store(startTs) + res.replicateTs.Store(math.MaxUint64) return res } func (t *tableSinkWrapper) start(ctx context.Context, startTs model.Ts) (err error) { - if t.replicateTs != 0 { + if t.replicateTs.Load() != math.MaxUint64 { log.Panic("The table sink has already started", zap.String("namespace", t.changefeed.Namespace), zap.String("changefeed", t.changefeed.ID), zap.Int64("tableID", t.tableID), zap.Uint64("startTs", startTs), - zap.Uint64("oldReplicateTs", t.replicateTs), + zap.Uint64("oldReplicateTs", t.replicateTs.Load()), ) } // FIXME(qupeng): it can be re-fetched later instead of fails. - if t.replicateTs, err = t.genReplicateTs(ctx); err != nil { + ts, err := t.genReplicateTs(ctx) + if err != nil { return errors.Trace(err) } + t.replicateTs.Store(ts) log.Info("Sink is started", zap.String("namespace", t.changefeed.Namespace), zap.String("changefeed", t.changefeed.ID), zap.Int64("tableID", t.tableID), zap.Uint64("startTs", startTs), - zap.Uint64("replicateTs", t.replicateTs), + zap.Uint64("replicateTs", ts), ) // This start ts maybe greater than the initial start ts of the table sink. @@ -378,14 +387,16 @@ func (t *tableSinkWrapper) checkTableSinkHealth() (err error) { // committed at downstream but we don't know. So we need to update `replicateTs` // of the table so that we can re-send those events later. func (t *tableSinkWrapper) restart(ctx context.Context) (err error) { - if t.replicateTs, err = t.genReplicateTs(ctx); err != nil { + ts, err := t.genReplicateTs(ctx) + if err != nil { return errors.Trace(err) } + t.replicateTs.Store(ts) log.Info("Sink is restarted", zap.String("namespace", t.changefeed.Namespace), zap.String("changefeed", t.changefeed.ID), zap.Int64("tableID", t.tableID), - zap.Uint64("replicateTs", t.replicateTs)) + zap.Uint64("replicateTs", ts)) return nil } diff --git a/cdc/processor/sinkmanager/table_sink_wrapper_test.go b/cdc/processor/sinkmanager/table_sink_wrapper_test.go index 9bee0b0ffba..84f98b0a115 100644 --- a/cdc/processor/sinkmanager/table_sink_wrapper_test.go +++ b/cdc/processor/sinkmanager/table_sink_wrapper_test.go @@ -51,6 +51,10 @@ func (m *mockSink) WriteEvents(events ...*eventsink.CallbackableEvent[*model.Row return nil } +func (m *mockSink) SchemeOption() (string, bool) { + return sink.BlackHoleScheme, false +} + func (m *mockSink) GetEvents() []*eventsink.CallbackableEvent[*model.RowChangedEvent] { m.mu.Lock() defer m.mu.Unlock() diff --git a/cdc/processor/sourcemanager/manager.go b/cdc/processor/sourcemanager/manager.go index 9916e4c68fa..2ec2b205ba3 100644 --- a/cdc/processor/sourcemanager/manager.go +++ b/cdc/processor/sourcemanager/manager.go @@ -14,11 +14,9 @@ package sourcemanager import ( - "context" "sync" "time" - "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" @@ -28,10 +26,7 @@ import ( "github.com/pingcap/tiflow/cdc/puller" cdccontext "github.com/pingcap/tiflow/pkg/context" cerrors "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/retry" "github.com/pingcap/tiflow/pkg/upstream" - "github.com/tikv/client-go/v2/oracle" - pd "github.com/tikv/pd/client" "go.uber.org/zap" ) @@ -56,9 +51,6 @@ type SourceManager struct { bdrMode bool safeModeAtStart bool - startTs model.Ts - - enableTableMonitor bool } // New creates a new source manager. @@ -69,55 +61,32 @@ func New( engine engine.SortEngine, errChan chan error, bdrMode bool, - enableTableMonitor bool, safeModeAtStart bool, ) *SourceManager { - startTs, err := getCurrentTs(context.Background(), up.PDClient) - if err != nil { - log.Panic("Cannot get current ts when creating source manager", - zap.String("namespace", changefeedID.Namespace), - zap.String("changefeed", changefeedID.ID)) - return nil - } return &SourceManager{ - changefeedID: changefeedID, - up: up, - mg: mg, - engine: engine, - errChan: errChan, - bdrMode: bdrMode, - enableTableMonitor: enableTableMonitor, - safeModeAtStart: safeModeAtStart, - startTs: startTs, + changefeedID: changefeedID, + up: up, + mg: mg, + engine: engine, + errChan: errChan, + bdrMode: bdrMode, + safeModeAtStart: safeModeAtStart, } } -func isOldUpdateKVEntry(raw *model.RawKVEntry, thresholdTs model.Ts) bool { - return raw != nil && raw.IsUpdate() && raw.CRTs < thresholdTs -} - -func splitUpdateKVEntry(raw *model.RawKVEntry) (*model.RawKVEntry, *model.RawKVEntry, error) { - if raw == nil { - return nil, nil, errors.New("nil event cannot be split") - } - deleteKVEntry := *raw - deleteKVEntry.Value = nil - - insertKVEntry := *raw - insertKVEntry.OldValue = nil - - return &deleteKVEntry, &insertKVEntry, nil +func isOldUpdateKVEntry(raw *model.RawKVEntry, getReplicaTs func() model.Ts) bool { + return raw != nil && raw.IsUpdate() && raw.CRTs < getReplicaTs() } // AddTable adds a table to the source manager. Start puller and register table to the engine. -func (m *SourceManager) AddTable(ctx cdccontext.Context, tableID model.TableID, tableName string, startTs model.Ts) { +func (m *SourceManager) AddTable(ctx cdccontext.Context, tableID model.TableID, tableName string, startTs model.Ts, getReplicaTs func() model.Ts) { // Add table to the engine first, so that the engine can receive the events from the puller. m.engine.AddTable(tableID) shouldSplitKVEntry := func(raw *model.RawKVEntry) bool { - return m.safeModeAtStart && isOldUpdateKVEntry(raw, m.startTs) + return m.safeModeAtStart && isOldUpdateKVEntry(raw, getReplicaTs) } - p := pullerwrapper.NewPullerWrapper(m.changefeedID, tableID, tableName, startTs, m.bdrMode, shouldSplitKVEntry, splitUpdateKVEntry) - p.Start(ctx, m.up, m.engine, m.errChan, m.enableTableMonitor) + p := pullerwrapper.NewPullerWrapper(m.changefeedID, tableID, tableName, startTs, m.bdrMode, shouldSplitKVEntry) + p.Start(ctx, m.up, m.engine, m.errChan) m.pullers.Store(tableID, p) } @@ -189,23 +158,3 @@ func (m *SourceManager) Close() error { zap.Duration("cost", time.Since(start))) return nil } - -func getCurrentTs(ctx context.Context, pdClient pd.Client) (model.Ts, error) { - backoffBaseDelayInMs := int64(100) - totalRetryDuration := 10 * time.Second - var replicateTs model.Ts - err := retry.Do(ctx, func() error { - phy, logic, err := pdClient.GetTS(ctx) - if err != nil { - return errors.Trace(err) - } - replicateTs = oracle.ComposeTS(phy, logic) - return nil - }, retry.WithBackoffBaseDelay(backoffBaseDelayInMs), - retry.WithTotalRetryDuratoin(totalRetryDuration), - retry.WithIsRetryableErr(cerrors.IsRetryableError)) - if err != nil { - return model.Ts(0), errors.Trace(err) - } - return replicateTs, nil -} diff --git a/cdc/processor/sourcemanager/puller/puller_wrapper.go b/cdc/processor/sourcemanager/puller/puller_wrapper.go index 19d00845641..5dd94bb212a 100644 --- a/cdc/processor/sourcemanager/puller/puller_wrapper.go +++ b/cdc/processor/sourcemanager/puller/puller_wrapper.go @@ -32,12 +32,6 @@ import ( "go.uber.org/zap" ) -// ShouldSplitKVEntry checks whether the raw kv entry should be splitted. -type ShouldSplitKVEntry func(raw *model.RawKVEntry) bool - -// SplitUpdateKVEntry splits the raw kv entry into a delete entry and an insert entry. -type SplitUpdateKVEntry func(raw *model.RawKVEntry) (*model.RawKVEntry, *model.RawKVEntry, error) - // Wrapper is a wrapper of puller used by source manager. type Wrapper struct { changefeed model.ChangeFeedID @@ -51,8 +45,7 @@ type Wrapper struct { wg sync.WaitGroup bdrMode bool - shouldSplitKVEntry ShouldSplitKVEntry - splitUpdateKVEntry SplitUpdateKVEntry + shouldSplitKVEntry model.ShouldSplitKVEntry } // NewPullerWrapper creates a new puller wrapper. @@ -62,8 +55,7 @@ func NewPullerWrapper( tableName string, startTs model.Ts, bdrMode bool, - shouldSplitKVEntry ShouldSplitKVEntry, - splitUpdateKVEntry SplitUpdateKVEntry, + shouldSplitKVEntry model.ShouldSplitKVEntry, ) *Wrapper { return &Wrapper{ changefeed: changefeed, @@ -72,7 +64,6 @@ func NewPullerWrapper( startTs: startTs, bdrMode: bdrMode, shouldSplitKVEntry: shouldSplitKVEntry, - splitUpdateKVEntry: splitUpdateKVEntry, } } @@ -91,7 +82,6 @@ func (n *Wrapper) Start( up *upstream.Upstream, eventSortEngine engine.SortEngine, errChan chan<- error, - enableTableMonitor bool, ) { failpoint.Inject("ProcessorAddTableError", func() { errChan <- cerrors.New("processor add table injected error") @@ -116,7 +106,6 @@ func (n *Wrapper) Start( n.tableID, n.tableName, n.bdrMode, - enableTableMonitor, ) n.wg.Add(1) go func() { @@ -148,7 +137,7 @@ func (n *Wrapper) Start( continue } if n.shouldSplitKVEntry(rawKV) { - deleteKVEntry, insertKVEntry, err := n.splitUpdateKVEntry(rawKV) + deleteKVEntry, insertKVEntry, err := model.SplitUpdateKVEntry(rawKV) if err != nil { log.Panic("failed to split update kv entry", zap.Error(err)) return diff --git a/cdc/puller/ddl_puller.go b/cdc/puller/ddl_puller.go index d29b131acc8..98a438c84fc 100644 --- a/cdc/puller/ddl_puller.go +++ b/cdc/puller/ddl_puller.go @@ -16,6 +16,7 @@ package puller import ( "context" "encoding/json" + "fmt" "sync" "sync/atomic" "time" @@ -28,6 +29,7 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tiflow/cdc/contextutil" "github.com/pingcap/tiflow/cdc/entry" + "github.com/pingcap/tiflow/cdc/entry/schema" "github.com/pingcap/tiflow/cdc/kv" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sorter/memory" @@ -122,8 +124,7 @@ func (p *ddlJobPullerImpl) Run(ctx context.Context) error { if job != nil { skip, err := p.handleJob(job) if err != nil { - return cerror.WrapError(cerror.ErrHandleDDLFailed, - err, job.String(), job.Query, job.StartTS, job.StartTS) + return err } log.Info("handle ddl job", zap.String("namespace", p.changefeedID.Namespace), @@ -358,7 +359,8 @@ func (p *ddlJobPullerImpl) handleJob(job *timodel.Job) (skip bool, err error) { if p.filter.ShouldDiscardDDL(job.Type, job.SchemaName, job.TableName) { return true, nil } - return true, errors.Trace(err) + return false, cerror.WrapError(cerror.ErrHandleDDLFailed, + errors.Trace(err), job.Query, job.StartTS, job.StartTS) } if job.BinlogInfo.FinishedTS <= p.getResolvedTs() || @@ -381,7 +383,17 @@ func (p *ddlJobPullerImpl) handleJob(job *timodel.Job) (skip bool, err error) { case timodel.ActionRenameTables: skip, err = p.handleRenameTables(job) if err != nil { - return true, errors.Trace(err) + log.Warn("handle rename tables ddl job failed", + zap.String("namespace", p.changefeedID.Namespace), + zap.String("changefeed", p.changefeedID.ID), + zap.String("schema", job.SchemaName), + zap.String("table", job.TableName), + zap.String("query", job.Query), + zap.Uint64("startTs", job.StartTS), + zap.Uint64("finishTs", job.BinlogInfo.FinishedTS), + zap.Error(err)) + return false, cerror.WrapError(cerror.ErrHandleDDLFailed, + errors.Trace(err), job.Query, job.StartTS, job.StartTS) } case timodel.ActionRenameTable: log.Info("rename table ddl job", @@ -396,7 +408,7 @@ func (p *ddlJobPullerImpl) handleJob(job *timodel.Job) (skip bool, err error) { // 1. If we can not find the old table, and the new table name is in filter rule, return error. discard := p.filter.ShouldDiscardDDL(job.Type, job.SchemaName, job.BinlogInfo.TableInfo.Name.O) if !discard { - return true, cerror.ErrSyncRenameTableFailed.GenWithStackByArgs(job.TableID, job.Query) + return false, cerror.ErrSyncRenameTableFailed.GenWithStackByArgs(job.TableID, job.Query) } skip = true } else { @@ -408,7 +420,7 @@ func (p *ddlJobPullerImpl) handleJob(job *timodel.Job) (skip bool, err error) { skipByNewTableName := p.filter.ShouldDiscardDDL(job.Type, job.SchemaName, job.BinlogInfo.TableInfo.Name.O) // 3. If its old table name is not in filter rule, and its new table name in filter rule, return error. if skipByOldTableName && !skipByNewTableName { - return true, cerror.ErrSyncRenameTableFailed.GenWithStackByArgs(job.TableID, job.Query) + return false, cerror.ErrSyncRenameTableFailed.GenWithStackByArgs(job.TableID, job.Query) } if skipByOldTableName && skipByNewTableName { skip = true @@ -437,13 +449,53 @@ func (p *ddlJobPullerImpl) handleJob(job *timodel.Job) (skip bool, err error) { zap.String("table", job.BinlogInfo.TableInfo.Name.O), zap.String("job", job.String()), zap.Error(err)) - return true, errors.Trace(err) + return false, cerror.WrapError(cerror.ErrHandleDDLFailed, + errors.Trace(err), job.Query, job.StartTS, job.StartTS) } p.setResolvedTs(job.BinlogInfo.FinishedTS) p.schemaVersion = job.BinlogInfo.SchemaVersion - return false, nil + return p.checkIneligibleTableDDL(snap, job) +} + +// checkIneligibleTableDDL checks if the table is ineligible before and after the DDL. +// 1. If it is not a table DDL, we shouldn't check it. +// 2. If the table after the DDL is ineligible: +// a. If the table is not exist before the DDL, we should ignore the DDL. +// b. If the table is ineligible before the DDL, we should ignore the DDL. +// c. If the table is eligible before the DDL, we should return an error. +func (p *ddlJobPullerImpl) checkIneligibleTableDDL(snapBefore *schema.Snapshot, job *timodel.Job) (skip bool, err error) { + if filter.IsSchemaDDL(job.Type) { + return false, nil + } + + ineligible := p.schemaStorage.GetLastSnapshot().IsIneligibleTableID(job.TableID) + if !ineligible { + return false, nil + } + + // If the table is not in the snapshot before the DDL, + // we should ignore the DDL. + _, exist := snapBefore.PhysicalTableByID(job.TableID) + if !exist { + return true, nil + } + + // If the table after the DDL is ineligible, we should check if it is not ineligible before the DDL. + // If so, we should return an error to inform the user that it is a + // dangerous operation and should be handled manually. + isBeforeineligible := snapBefore.IsIneligibleTableID(job.TableID) + if isBeforeineligible { + log.Warn("ignore the DDL event of ineligible table", + zap.String("changefeed", p.changefeedID.ID), zap.Any("ddl", job)) + return true, nil + } + return false, cerror.New(fmt.Sprintf("An eligible table become ineligible after DDL: [%s] "+ + "it is a dangerous operation and may cause data loss. If you want to replicate this ddl safely, "+ + "pelase pause the changefeed and update the `force-replicate=true` "+ + "in the changefeed configuration, "+ + "then resume the changefeed.", job.Query)) } func findDBByName(dbs []*timodel.DBInfo, name string) (*timodel.DBInfo, error) { @@ -511,7 +563,6 @@ func NewDDLJobPuller( changefeed, -1, DDLPullerTableName, ddLPullerFilterLoop, - false, ), kvStorage: kvStorage, outputCh: make(chan *model.DDLJobEntry, defaultPullerOutputChanSize), diff --git a/cdc/puller/ddl_puller_test.go b/cdc/puller/ddl_puller_test.go index aa51bf1ef6b..b5a3ffa719c 100644 --- a/cdc/puller/ddl_puller_test.go +++ b/cdc/puller/ddl_puller_test.go @@ -199,23 +199,23 @@ func TestHandleRenameTable(t *testing.T) { mockPuller.appendResolvedTs(job.BinlogInfo.FinishedTS + 1) waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) - job = helper.DDL2Job("create table test1.t1(id int)") + job = helper.DDL2Job("create table test1.t1(id int primary key)") remainTables[0] = job.TableID mockPuller.appendDDL(job) mockPuller.appendResolvedTs(job.BinlogInfo.FinishedTS + 1) waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) - job = helper.DDL2Job("create table test1.t2(id int)") + job = helper.DDL2Job("create table test1.t2(id int primary key)") mockPuller.appendDDL(job) mockPuller.appendResolvedTs(job.BinlogInfo.FinishedTS + 1) waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) - job = helper.DDL2Job("create table test1.t3(id int)") + job = helper.DDL2Job("create table test1.t3(id int primary key)") mockPuller.appendDDL(job) mockPuller.appendResolvedTs(job.BinlogInfo.FinishedTS + 1) waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) - job = helper.DDL2Job("create table test1.t5(id int)") + job = helper.DDL2Job("create table test1.t5(id int primary key)") mockPuller.appendDDL(job) mockPuller.appendResolvedTs(job.BinlogInfo.FinishedTS + 1) waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) @@ -225,7 +225,7 @@ func TestHandleRenameTable(t *testing.T) { mockPuller.appendResolvedTs(job.BinlogInfo.FinishedTS + 1) waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) - job = helper.DDL2Job("create table ignore1.a(id int)") + job = helper.DDL2Job("create table ignore1.a(id int primary key)") mockPuller.appendDDL(job) mockPuller.appendResolvedTs(job.BinlogInfo.FinishedTS + 1) waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) @@ -240,7 +240,7 @@ func TestHandleRenameTable(t *testing.T) { } { - _ = helper.DDL2Job("create table test1.t6(id int)") + _ = helper.DDL2Job("create table test1.t6(id int primary key)") job := helper.DDL2Job("rename table test1.t2 to test1.t22, test1.t6 to test1.t66") skip, err := ddlJobPullerImpl.handleRenameTables(job) require.Error(t, err) @@ -257,17 +257,17 @@ func TestHandleRenameTable(t *testing.T) { mockPuller.appendResolvedTs(job.BinlogInfo.FinishedTS + 1) waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) - job = helper.DDL2Job("create table test2.t1(id int)") + job = helper.DDL2Job("create table test2.t1(id int primary key)") mockPuller.appendDDL(job) mockPuller.appendResolvedTs(job.BinlogInfo.FinishedTS + 1) waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) - job = helper.DDL2Job("create table test2.t2(id int)") + job = helper.DDL2Job("create table test2.t2(id int primary key)") mockPuller.appendDDL(job) mockPuller.appendResolvedTs(job.BinlogInfo.FinishedTS + 1) waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) - job = helper.DDL2Job("create table test2.t3(id int)") + job = helper.DDL2Job("create table test2.t3(id int primary key)") mockPuller.appendDDL(job) mockPuller.appendResolvedTs(job.BinlogInfo.FinishedTS + 1) waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) @@ -285,13 +285,13 @@ func TestHandleRenameTable(t *testing.T) { mockPuller.appendResolvedTs(job.BinlogInfo.FinishedTS + 1) waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) - job = helper.DDL2Job("create table Test3.t1(id int)") + job = helper.DDL2Job("create table Test3.t1(id int primary key)") mockPuller.appendDDL(job) mockPuller.appendResolvedTs(job.BinlogInfo.FinishedTS + 1) waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) // skip this table - job = helper.DDL2Job("create table Test3.t2(id int)") + job = helper.DDL2Job("create table Test3.t2(id int primary key)") mockPuller.appendDDL(job) mockPuller.appendResolvedTs(job.BinlogInfo.FinishedTS + 1) waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) @@ -307,34 +307,34 @@ func TestHandleRenameTable(t *testing.T) { // test rename table { - job := helper.DDL2Job("create table test1.t99 (id int)") + job := helper.DDL2Job("create table test1.t99 (id int primary key)") mockPuller.appendDDL(job) mockPuller.appendResolvedTs(job.BinlogInfo.FinishedTS + 1) waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) // this ddl should be skipped - job = helper.DDL2Job("create table test1.t1000 (id int)") + job = helper.DDL2Job("create table test1.t1000 (id int primary key)") mockPuller.appendDDL(job) mockPuller.appendResolvedTs(job.BinlogInfo.FinishedTS + 1) waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) // this ddl should be skipped - job = helper.DDL2Job("create table test1.t888 (id int)") + job = helper.DDL2Job("create table test1.t888 (id int primary key)") mockPuller.appendDDL(job) mockPuller.appendResolvedTs(job.BinlogInfo.FinishedTS + 1) waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) - job = helper.DDL2Job("create table test1.t20230808 (id int)") + job = helper.DDL2Job("create table test1.t20230808 (id int primary key)") mockPuller.appendDDL(job) mockPuller.appendResolvedTs(job.BinlogInfo.FinishedTS + 1) waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) - job = helper.DDL2Job("create table test1.t202308081 (id int)") + job = helper.DDL2Job("create table test1.t202308081 (id int primary key)") mockPuller.appendDDL(job) mockPuller.appendResolvedTs(job.BinlogInfo.FinishedTS + 1) waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) - job = helper.DDL2Job("create table test1.t202308082 (id int)") + job = helper.DDL2Job("create table test1.t202308082 (id int primary key)") mockPuller.appendDDL(job) mockPuller.appendResolvedTs(job.BinlogInfo.FinishedTS + 1) waitResolvedTs(t, ddlJobPuller, job.BinlogInfo.FinishedTS+1) @@ -346,9 +346,8 @@ func TestHandleRenameTable(t *testing.T) { // since test1.t100 is in filter rule, replicate it job = helper.DDL2Job("rename table test1.t1000 to test1.t100") - skip, err = ddlJobPullerImpl.handleJob(job) + _, err = ddlJobPullerImpl.handleJob(job) require.Error(t, err) - require.True(t, skip) require.Contains(t, err.Error(), fmt.Sprintf("table's old name is not in filter rule, and its new name in filter rule "+ "table id '%d', ddl query: [%s], it's an unexpected behavior, "+ "if you want to replicate this table, please add its old name to filter rule.", job.TableID, job.Query)) @@ -370,9 +369,8 @@ func TestHandleRenameTable(t *testing.T) { // but now it will throw an error since schema ignore1 are not in schemaStorage // ref: https://github.com/pingcap/tiflow/issues/9488 job = helper.DDL2Job("rename table test1.t202308081 to ignore1.ignore1, test1.t202308082 to ignore1.dongmen") - skip, err = ddlJobPullerImpl.handleJob(job) + _, err = ddlJobPullerImpl.handleJob(job) require.NotNil(t, err) - require.True(t, skip) require.Contains(t, err.Error(), "ErrSnapshotSchemaNotFound") } } @@ -432,7 +430,7 @@ func TestHandleJob(t *testing.T) { // test create table { - job := helper.DDL2Job("create table test1.t1(id int) partition by range(id) (partition p0 values less than (10))") + job := helper.DDL2Job("create table test1.t1(id int primary key) partition by range(id) (partition p0 values less than (10))") skip, err := ddlJobPullerImpl.handleJob(job) require.NoError(t, err) require.False(t, skip) @@ -442,7 +440,7 @@ func TestHandleJob(t *testing.T) { require.NoError(t, err) require.False(t, skip) - job = helper.DDL2Job("create table test1.testStartTs(id int)") + job = helper.DDL2Job("create table test1.testStartTs(id int primary key)") skip, err = ddlJobPullerImpl.handleJob(job) require.NoError(t, err) require.False(t, skip) @@ -453,23 +451,23 @@ func TestHandleJob(t *testing.T) { require.NoError(t, err) require.False(t, skip) - job = helper.DDL2Job("create table test1.t2(id int)") + job = helper.DDL2Job("create table test1.t2(id int primary key)") skip, err = ddlJobPullerImpl.handleJob(job) require.NoError(t, err) require.False(t, skip) - job = helper.DDL2Job("create table test1.t3(id int)") + job = helper.DDL2Job("create table test1.t3(id int primary key)") skip, err = ddlJobPullerImpl.handleJob(job) require.NoError(t, err) require.True(t, skip) - job = helper.DDL2Job("create table test1.t4(id int) partition by range(id) (partition p0 values less than (10))") + job = helper.DDL2Job("create table test1.t4(id int primary key) partition by range(id) (partition p0 values less than (10))") skip, err = ddlJobPullerImpl.handleJob(job) require.NoError(t, err) require.True(t, skip) // make sure no schema not found error - job = helper.DDL2Job("create table test3.t1(id int) partition by range(id) (partition p0 values less than (10))") + job = helper.DDL2Job("create table test3.t1(id int primary key) partition by range(id) (partition p0 values less than (10))") skip, err = ddlJobPullerImpl.handleJob(job) require.NoError(t, err) require.True(t, skip) @@ -645,7 +643,7 @@ func TestDDLPuller(t *testing.T) { StartTS: 5, State: timodel.JobStateDone, BinlogInfo: &timodel.HistoryInfo{SchemaVersion: 2, FinishedTS: 18}, - Query: "create table test.t1(id int)", + Query: "create table test.t1(id int primary key)", }) mockPuller.appendDDL(&timodel.Job{ ID: 1, @@ -653,7 +651,7 @@ func TestDDLPuller(t *testing.T) { StartTS: 5, State: timodel.JobStateDone, BinlogInfo: &timodel.HistoryInfo{SchemaVersion: 1, FinishedTS: 16}, - Query: "create table t2(id int)", + Query: "create table t2(id int primary key)", }) resolvedTs, ddl = p.PopFrontDDL() require.Equal(t, resolvedTs, uint64(15)) @@ -678,7 +676,7 @@ func TestDDLPuller(t *testing.T) { StartTS: 20, State: timodel.JobStateDone, BinlogInfo: &timodel.HistoryInfo{SchemaVersion: 4, FinishedTS: 25}, - Query: "create table t3(id int)", + Query: "create table t3(id int primary key)", }) mockPuller.appendDDL(&timodel.Job{ ID: 3, @@ -686,7 +684,7 @@ func TestDDLPuller(t *testing.T) { StartTS: 20, State: timodel.JobStateDone, BinlogInfo: &timodel.HistoryInfo{SchemaVersion: 4, FinishedTS: 25}, - Query: "create table t3(id int)", + Query: "create table t3(id int primary key)", }) mockPuller.appendResolvedTs(30) waitResolvedTsGrowing(t, p, 25) @@ -708,7 +706,7 @@ func TestDDLPuller(t *testing.T) { StartTS: 20, State: timodel.JobStateCancelled, BinlogInfo: &timodel.HistoryInfo{SchemaVersion: 6, FinishedTS: 36}, - Query: "create table t4(id int)", + Query: "create table t4(id int primary key)", }) mockPuller.appendResolvedTs(40) waitResolvedTsGrowing(t, p, 40) @@ -803,3 +801,65 @@ func waitResolvedTsGrowing(t *testing.T, p DDLPuller, targetTs model.Ts) { }, retry.WithBackoffBaseDelay(20), retry.WithMaxTries(200)) require.Nil(t, err) } + +func TestCcheckIneligibleTableDDL(t *testing.T) { + startTs := uint64(10) + mockPuller := newMockPuller(t, startTs) + ddlJobPuller, helper := newMockDDLJobPuller(t, mockPuller, true) + defer helper.Close() + + ddlJobPullerImpl := ddlJobPuller.(*ddlJobPullerImpl) + ddlJobPullerImpl.setResolvedTs(startTs) + + cfg := config.GetDefaultReplicaConfig() + f, err := filter.NewFilter(cfg, "") + require.NoError(t, err) + ddlJobPullerImpl.filter = f + + ddl := helper.DDL2Job("CREATE DATABASE test1") + skip, err := ddlJobPullerImpl.handleJob(ddl) + require.NoError(t, err) + require.False(t, skip) + + // case 1: create a table only has a primary key and drop it, expect an error. + // It is because the table is not eligible after the drop primary key DDL. + ddl = helper.DDL2Job(`CREATE TABLE test1.t1 ( + id INT PRIMARY KEY /*T![clustered_index] NONCLUSTERED */, + name VARCHAR(255), + email VARCHAR(255) UNIQUE + );`) + skip, err = ddlJobPullerImpl.handleJob(ddl) + require.NoError(t, err) + require.False(t, skip) + + ddl = helper.DDL2Job("ALTER TABLE test1.t1 DROP PRIMARY KEY;") + skip, err = ddlJobPullerImpl.handleJob(ddl) + require.Error(t, err) + require.False(t, skip) + require.Contains(t, err.Error(), "An eligible table become ineligible after DDL") + + // case 2: create a table has a primary key and another not null unique key, + // and drop the primary key, expect no error. + // It is because the table is still eligible after the drop primary key DDL. + ddl = helper.DDL2Job(`CREATE TABLE test1.t2 ( + id INT PRIMARY KEY /*T![clustered_index] NONCLUSTERED */, + name VARCHAR(255), + email VARCHAR(255) NOT NULL UNIQUE + );`) + skip, err = ddlJobPullerImpl.handleJob(ddl) + require.NoError(t, err) + require.False(t, skip) + + ddl = helper.DDL2Job("ALTER TABLE test1.t2 DROP PRIMARY KEY;") + skip, err = ddlJobPullerImpl.handleJob(ddl) + require.NoError(t, err) + require.False(t, skip) + + // case 3: continue to drop the unique key, expect an error. + // It is because the table is not eligible after the drop unique key DDL. + ddl = helper.DDL2Job("ALTER TABLE test1.t2 DROP INDEX email;") + skip, err = ddlJobPullerImpl.handleJob(ddl) + require.Error(t, err) + require.False(t, skip) + require.Contains(t, err.Error(), "An eligible table become ineligible after DDL") +} diff --git a/cdc/puller/puller.go b/cdc/puller/puller.go index 930571dddb3..a89559f5445 100644 --- a/cdc/puller/puller.go +++ b/cdc/puller/puller.go @@ -80,8 +80,6 @@ type pullerImpl struct { lastForwardResolvedTs uint64 // startResolvedTs is the resolvedTs when puller is initialized startResolvedTs uint64 - - enableTableMonitor bool } // New create a new Puller fetch event start from checkpointTs and put into buf. @@ -98,7 +96,6 @@ func New(ctx context.Context, tableID model.TableID, tableName string, filterLoop bool, - enableTableMonitor bool, ) Puller { tikvStorage, ok := kvStorage.(tikv.Storage) if !ok { @@ -133,8 +130,7 @@ func New(ctx context.Context, tableName: tableName, cfg: cfg, - startResolvedTs: checkpointTs, - enableTableMonitor: enableTableMonitor, + startResolvedTs: checkpointTs, } return p } @@ -152,7 +148,7 @@ func (p *pullerImpl) Run(ctx context.Context) error { span := span g.Go(func() error { - return p.kvCli.EventFeed(ctx, span, checkpointTs, lockResolver, eventCh, p.enableTableMonitor) + return p.kvCli.EventFeed(ctx, span, checkpointTs, lockResolver, eventCh) }) } diff --git a/cdc/puller/puller_test.go b/cdc/puller/puller_test.go index 5f888e89848..408a86ba903 100644 --- a/cdc/puller/puller_test.go +++ b/cdc/puller/puller_test.go @@ -79,7 +79,6 @@ func (mc *mockCDCKVClient) EventFeed( ts uint64, lockResolver txnutil.LockResolver, eventCh chan<- model.RegionFeedEvent, - enableTableMonitor bool, ) error { for { select { @@ -132,7 +131,7 @@ func newPullerForTest( plr := New( ctx, pdCli, grpcPool, regionCache, store, pdutil.NewClock4Test(), checkpointTs, spans, config.GetDefaultServerConfig(), - model.DefaultChangeFeedID("changefeed-id-test"), 0, "table-test", false, false) + model.DefaultChangeFeedID("changefeed-id-test"), 0, "table-test", false) wg.Add(1) go func() { defer wg.Done() diff --git a/cdc/redo/reader/reader.go b/cdc/redo/reader/reader.go index 71c641099ca..9959a7212b9 100644 --- a/cdc/redo/reader/reader.go +++ b/cdc/redo/reader/reader.go @@ -37,7 +37,7 @@ import ( const ( emitBatch = mysql.DefaultMaxTxnRow defaultReaderChanSize = mysql.DefaultWorkerCount * emitBatch - maxTotalMemoryUsage = 90.0 + maxTotalMemoryUsage = 80.0 maxWaitDuration = time.Minute * 2 ) @@ -221,11 +221,6 @@ func (l *LogReader) runReader(egCtx context.Context, cfg *readerConfig) error { case l.rowCh <- row: } } - err := util.WaitMemoryAvailable(maxTotalMemoryUsage, maxWaitDuration) - if err != nil { - return errors.Trace(err) - } - case redo.RedoDDLLogFileType: ddl := model.LogToDDL(item.data.RedoDDL) if ddl != nil && ddl.CommitTs > cfg.startTs && ddl.CommitTs <= cfg.endTs { @@ -364,6 +359,7 @@ func (h logHeap) Len() int { } func (h logHeap) Less(i, j int) bool { + // we separate ddl and dml, so we only need to compare dml with dml, and ddl with ddl. if h[i].data.Type == model.RedoLogTypeDDL { if h[i].data.RedoDDL == nil || h[i].data.RedoDDL.DDL == nil { return true @@ -381,10 +377,19 @@ func (h logHeap) Less(i, j int) bool { return false } - if h[i].data.RedoRow.Row.CommitTs == h[j].data.RedoRow.Row.CommitTs && - h[i].data.RedoRow.Row.StartTs < h[j].data.RedoRow.Row.StartTs { - return true + if h[i].data.RedoRow.Row.CommitTs == h[j].data.RedoRow.Row.CommitTs { + if h[i].data.RedoRow.Row.StartTs != h[j].data.RedoRow.Row.StartTs { + return h[i].data.RedoRow.Row.StartTs < h[j].data.RedoRow.Row.StartTs + } + // in the same txn, we need to sort by delete/update/insert order + if h[i].data.RedoRow.Row.IsDelete() { + return true + } else if h[i].data.RedoRow.Row.IsUpdate() { + return !h[j].data.RedoRow.Row.IsDelete() + } + return false } + return h[i].data.RedoRow.Row.CommitTs < h[j].data.RedoRow.Row.CommitTs } diff --git a/cdc/redo/reader/reader_test.go b/cdc/redo/reader/reader_test.go index 4abb597e048..219eb975c2a 100644 --- a/cdc/redo/reader/reader_test.go +++ b/cdc/redo/reader/reader_test.go @@ -243,3 +243,515 @@ func genMetaFile(t *testing.T, dir string, meta *common.LogMeta) { _, err = f.Write(data) require.Nil(t, err) } + +func TestLogHeapLess(t *testing.T) { + tests := []struct { + name string + h logHeap + i int + j int + expect bool + }{ + { + name: "Delete before Update", + h: logHeap{ + { + data: &model.RedoLog{ + Type: model.RedoLogTypeRow, + RedoRow: &model.RedoRowChangedEvent{ + Row: &model.RowChangedEvent{ + CommitTs: 100, + Table: &model.TableName{ + Schema: "test", + Table: "table", + TableID: 1, + IsPartition: false, + }, + PreColumns: []*model.Column{ + { + Name: "col-1", + Value: 1, + }, { + Name: "col-2", + Value: 2, + }, + }, + }, + }, + }, + }, + { + data: &model.RedoLog{ + Type: model.RedoLogTypeRow, + RedoRow: &model.RedoRowChangedEvent{ + Row: &model.RowChangedEvent{ + CommitTs: 100, + Table: &model.TableName{ + Schema: "test", + Table: "table", + TableID: 1, + IsPartition: false, + }, + PreColumns: []*model.Column{ + { + Name: "col-1", + Value: 1, + }, { + Name: "col-2", + Value: 2, + }, + }, + Columns: []*model.Column{ + { + Name: "col-1", + Value: 1, + }, { + Name: "col-2", + Value: 3, + }, + }, + }, + }, + }, + }, + }, + i: 0, + j: 1, + expect: true, + }, + { + name: "Update before Insert", + h: logHeap{ + { + data: &model.RedoLog{ + Type: model.RedoLogTypeRow, + RedoRow: &model.RedoRowChangedEvent{ + Row: &model.RowChangedEvent{ + CommitTs: 100, + Table: &model.TableName{ + Schema: "test", + Table: "table", + TableID: 1, + IsPartition: false, + }, + PreColumns: []*model.Column{ + { + Name: "col-1", + Value: 1, + }, { + Name: "col-2", + Value: 2, + }, + }, + Columns: []*model.Column{ + { + Name: "col-1", + Value: 1, + }, { + Name: "col-2", + Value: 3, + }, + }, + }, + }, + }, + }, + { + data: &model.RedoLog{ + Type: model.RedoLogTypeRow, + RedoRow: &model.RedoRowChangedEvent{ + Row: &model.RowChangedEvent{ + CommitTs: 100, + Table: &model.TableName{ + Schema: "test", + Table: "table", + TableID: 1, + IsPartition: false, + }, + Columns: []*model.Column{ + { + Name: "col-1", + Value: 1, + }, { + Name: "col-2", + Value: 1, + }, + }, + }, + }, + }, + }, + }, + i: 0, + j: 1, + expect: true, + }, + { + name: "Update before Delete", + h: logHeap{ + { + data: &model.RedoLog{ + Type: model.RedoLogTypeRow, + RedoRow: &model.RedoRowChangedEvent{ + Row: &model.RowChangedEvent{ + CommitTs: 100, + Table: &model.TableName{ + Schema: "test", + Table: "table", + TableID: 1, + IsPartition: false, + }, + PreColumns: []*model.Column{ + { + Name: "col-1", + Value: 1, + }, { + Name: "col-2", + Value: 2, + }, + }, + Columns: []*model.Column{ + { + Name: "col-1", + Value: 1, + }, { + Name: "col-2", + Value: 3, + }, + }, + }, + }, + }, + }, + { + data: &model.RedoLog{ + Type: model.RedoLogTypeRow, + RedoRow: &model.RedoRowChangedEvent{ + Row: &model.RowChangedEvent{ + CommitTs: 100, + Table: &model.TableName{ + Schema: "test", + Table: "table", + TableID: 1, + IsPartition: false, + }, + PreColumns: []*model.Column{ + { + Name: "col-1", + Value: 1, + }, { + Name: "col-2", + Value: 1, + }, + }, + }, + }, + }, + }, + }, + i: 0, + j: 1, + expect: false, + }, + { + name: "Update before Update", + h: logHeap{ + { + data: &model.RedoLog{ + Type: model.RedoLogTypeRow, + RedoRow: &model.RedoRowChangedEvent{ + Row: &model.RowChangedEvent{ + CommitTs: 100, + Table: &model.TableName{ + Schema: "test", + Table: "table", + TableID: 1, + IsPartition: false, + }, + PreColumns: []*model.Column{ + { + Name: "col-1", + Value: 1, + }, { + Name: "col-2", + Value: 2, + }, + }, + Columns: []*model.Column{ + { + Name: "col-1", + Value: 1, + }, { + Name: "col-2", + Value: 3, + }, + }, + }, + }, + }, + }, + { + data: &model.RedoLog{ + Type: model.RedoLogTypeRow, + RedoRow: &model.RedoRowChangedEvent{ + Row: &model.RowChangedEvent{ + CommitTs: 100, + Table: &model.TableName{ + Schema: "test", + Table: "table", + TableID: 1, + IsPartition: false, + }, + PreColumns: []*model.Column{ + { + Name: "col-1", + Value: 1, + }, { + Name: "col-2", + Value: 1, + }, + }, + Columns: []*model.Column{ + { + Name: "col-1", + Value: 1, + }, { + Name: "col-2", + Value: 4, + }, + }, + }, + }, + }, + }, + }, + i: 0, + j: 1, + expect: true, + }, + { + name: "Delete before Insert", + h: logHeap{ + { + data: &model.RedoLog{ + Type: model.RedoLogTypeRow, + RedoRow: &model.RedoRowChangedEvent{ + Row: &model.RowChangedEvent{ + CommitTs: 100, + Table: &model.TableName{ + Schema: "test", + Table: "table", + TableID: 1, + IsPartition: false, + }, + PreColumns: []*model.Column{ + { + Name: "col-1", + Value: 1, + }, { + Name: "col-2", + Value: 1, + }, + }, + }, + }, + }, + }, + { + data: &model.RedoLog{ + Type: model.RedoLogTypeRow, + RedoRow: &model.RedoRowChangedEvent{ + Row: &model.RowChangedEvent{ + CommitTs: 100, + Table: &model.TableName{ + Schema: "test", + Table: "table", + TableID: 1, + IsPartition: false, + }, + Columns: []*model.Column{ + { + Name: "col-1", + Value: 1, + }, { + Name: "col-2", + Value: 3, + }, + }, + }, + }, + }, + }, + }, + i: 0, + j: 1, + expect: true, + }, + { + name: "Same type of operations, different commit ts", + h: logHeap{ + { + data: &model.RedoLog{ + Type: model.RedoLogTypeRow, + RedoRow: &model.RedoRowChangedEvent{ + Row: &model.RowChangedEvent{ + CommitTs: 100, + Table: &model.TableName{ + Schema: "test", + Table: "table", + TableID: 1, + IsPartition: false, + }, + }, + }, + }, + }, + { + data: &model.RedoLog{ + Type: model.RedoLogTypeRow, + RedoRow: &model.RedoRowChangedEvent{ + Row: &model.RowChangedEvent{ + CommitTs: 200, + Table: &model.TableName{ + Schema: "test", + Table: "table", + TableID: 1, + IsPartition: false, + }, + }, + }, + }, + }, + }, + i: 0, + j: 1, + expect: true, + }, + { + name: "Same type of operations, same commit ts, different startTs", + h: logHeap{ + { + data: &model.RedoLog{ + Type: model.RedoLogTypeRow, + RedoRow: &model.RedoRowChangedEvent{ + Row: &model.RowChangedEvent{ + CommitTs: 100, + StartTs: 80, + Table: &model.TableName{ + Schema: "test", + Table: "table", + TableID: 1, + IsPartition: false, + }, + }, + }, + }, + }, + { + data: &model.RedoLog{ + Type: model.RedoLogTypeRow, + RedoRow: &model.RedoRowChangedEvent{ + Row: &model.RowChangedEvent{ + CommitTs: 100, + StartTs: 90, + Table: &model.TableName{ + Schema: "test", + Table: "table", + TableID: 1, + IsPartition: false, + }, + }, + }, + }, + }, + }, + i: 0, + j: 1, + expect: true, + }, + { + name: "Same type of operations, same commit ts", + h: logHeap{ + { + data: &model.RedoLog{ + Type: model.RedoLogTypeRow, + RedoRow: &model.RedoRowChangedEvent{ + Row: &model.RowChangedEvent{ + CommitTs: 100, + Table: &model.TableName{ + Schema: "test", + Table: "table", + TableID: 1, + IsPartition: false, + }, + PreColumns: []*model.Column{ + { + Name: "col-1", + Value: 1, + }, { + Name: "col-2", + Value: 2, + }, + }, + Columns: []*model.Column{ + { + Name: "col-1", + Value: 1, + }, { + Name: "col-2", + Value: 3, + }, + }, + }, + }, + }, + }, + { + data: &model.RedoLog{ + Type: model.RedoLogTypeRow, + RedoRow: &model.RedoRowChangedEvent{ + Row: &model.RowChangedEvent{ + CommitTs: 100, + Table: &model.TableName{ + Schema: "test", + Table: "table", + TableID: 1, + IsPartition: false, + }, + PreColumns: []*model.Column{ + { + Name: "col-1", + Value: 1, + }, { + Name: "col-2", + Value: 1, + }, + }, + Columns: []*model.Column{ + { + Name: "col-1", + Value: 1, + }, { + Name: "col-2", + Value: 3, + }, + }, + }, + }, + }, + }, + }, + i: 0, + j: 1, + expect: true, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + if got := tt.h.Less(tt.i, tt.j); got != tt.expect { + t.Errorf("logHeap.Less() = %v, want %v", got, tt.expect) + } + }) + } +} diff --git a/cdc/redo/writer/memory/encoding_worker.go b/cdc/redo/writer/memory/encoding_worker.go index 883ebb13314..8930e7969e7 100644 --- a/cdc/redo/writer/memory/encoding_worker.go +++ b/cdc/redo/writer/memory/encoding_worker.go @@ -103,7 +103,7 @@ type encodingWorkerGroup struct { workerNum int nextWorker atomic.Uint64 - closed chan struct{} + closed chan error } func newEncodingWorkerGroup(cfg *writer.LogWriterConfig) *encodingWorkerGroup { @@ -120,19 +120,20 @@ func newEncodingWorkerGroup(cfg *writer.LogWriterConfig) *encodingWorkerGroup { inputChs: inputChs, outputCh: make(chan *polymorphicRedoEvent, redo.DefaultEncodingOutputChanSize), workerNum: workerNum, - closed: make(chan struct{}), + closed: make(chan error, 1), } } func (e *encodingWorkerGroup) Run(ctx context.Context) (err error) { defer func() { - close(e.closed) + log.Warn("redo encoding workers closed", + zap.String("namespace", e.changefeed.Namespace), + zap.String("changefeed", e.changefeed.ID), + zap.Error(err)) if err != nil && errors.Cause(err) != context.Canceled { - log.Warn("redo fileWorkerGroup closed with error", - zap.String("namespace", e.changefeed.Namespace), - zap.String("changefeed", e.changefeed.ID), - zap.Error(err)) + e.closed <- err } + close(e.closed) }() eg, egCtx := errgroup.WithContext(ctx) for i := 0; i < e.workerNum; i++ { @@ -182,8 +183,8 @@ func (e *encodingWorkerGroup) input( select { case <-ctx.Done(): return ctx.Err() - case <-e.closed: - return errors.ErrRedoWriterStopped.GenWithStackByArgs("encoding worker is closed") + case err := <-e.closed: + return errors.WrapError(errors.ErrRedoWriterStopped, err, "encoding worker is closed") case e.inputChs[idx] <- event: return nil } @@ -195,8 +196,8 @@ func (e *encodingWorkerGroup) output( select { case <-ctx.Done(): return ctx.Err() - case <-e.closed: - return errors.ErrRedoWriterStopped.GenWithStackByArgs("encoding worker is closed") + case err := <-e.closed: + return errors.WrapError(errors.ErrRedoWriterStopped, err, "encoding worker is closed") case e.outputCh <- event: return nil } @@ -220,8 +221,8 @@ func (e *encodingWorkerGroup) FlushAll(ctx context.Context) error { select { case <-ctx.Done(): return ctx.Err() - case <-e.closed: - return errors.ErrRedoWriterStopped.GenWithStackByArgs("encoding worker is closed") + case err := <-e.closed: + return errors.WrapError(errors.ErrRedoWriterStopped, err, "encoding worker is closed") case <-flushCh: } return nil @@ -244,8 +245,8 @@ func (e *encodingWorkerGroup) broadcastAndWaitEncoding(ctx context.Context) erro select { case <-ctx.Done(): return ctx.Err() - case <-e.closed: - return errors.ErrRedoWriterStopped.GenWithStackByArgs("encoding worker is closed") + case err := <-e.closed: + return errors.WrapError(errors.ErrRedoWriterStopped, err, "encoding worker is closed") case <-ch: } } diff --git a/cdc/redo/writer/memory/file_worker.go b/cdc/redo/writer/memory/file_worker.go index d566ddf66c5..2e8fe622de3 100644 --- a/cdc/redo/writer/memory/file_worker.go +++ b/cdc/redo/writer/memory/file_worker.go @@ -155,12 +155,10 @@ func (f *fileWorkerGroup) Run( ) (err error) { defer func() { f.close() - if err != nil && errors.Cause(err) != context.Canceled { - log.Warn("redo file workers closed with error", - zap.String("namespace", f.cfg.ChangeFeedID.Namespace), - zap.String("changefeed", f.cfg.ChangeFeedID.ID), - zap.Error(err)) - } + log.Warn("redo file workers closed", + zap.String("namespace", f.cfg.ChangeFeedID.Namespace), + zap.String("changefeed", f.cfg.ChangeFeedID.ID), + zap.Error(err)) }() eg, egCtx := errgroup.WithContext(ctx) diff --git a/cdc/redo/writer/memory/mem_log_writer_test.go b/cdc/redo/writer/memory/mem_log_writer_test.go index 53943e52a29..d3b72541f78 100644 --- a/cdc/redo/writer/memory/mem_log_writer_test.go +++ b/cdc/redo/writer/memory/mem_log_writer_test.go @@ -95,11 +95,9 @@ func testWriteEvents(t *testing.T, events []writer.RedoEvent) { require.NoError(t, err) require.ErrorIs(t, lw.Close(), context.Canceled) - require.Eventually(t, func() bool { - err = lw.WriteEvents(ctx, events...) - return err != nil - }, 2*time.Second, 10*time.Millisecond) - require.ErrorContains(t, err, "redo log writer stopped") + + err = lw.WriteEvents(ctx, events...) + require.NoError(t, err) err = lw.FlushLog(ctx) - require.ErrorContains(t, err, "redo log writer stopped") + require.NoError(t, err) } diff --git a/cdc/sink/mysql/mysql_syncpoint_store.go b/cdc/sink/mysql/mysql_syncpoint_store.go index 8c48c64a8e1..26cf286cde4 100644 --- a/cdc/sink/mysql/mysql_syncpoint_store.go +++ b/cdc/sink/mysql/mysql_syncpoint_store.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tiflow/pkg/errorutil" "github.com/pingcap/tiflow/pkg/filter" "github.com/pingcap/tiflow/pkg/security" + pmysql "github.com/pingcap/tiflow/pkg/sink/mysql" "go.uber.org/zap" ) @@ -120,11 +121,13 @@ func newMySQLSyncPointStore( if dsn.Params == nil { dsn.Params = make(map[string]string, 1) } - testDB, err := sql.Open("mysql", dsn.FormatDSN()) + + testDB, err := pmysql.GetTestDB(ctx, dsn, GetDBConnImpl) if err != nil { - return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection when configuring sink") + return nil, err } defer testDB.Close() + dsnStr, err = generateDSNByParams(ctx, dsn, params, testDB) if err != nil { return nil, errors.Trace(err) @@ -153,7 +156,7 @@ func (s *mysqlSyncPointStore) CreateSyncTable(ctx context.Context) error { tx, err := s.db.BeginTx(ctx, nil) if err != nil { log.Error("create sync table: begin Tx fail", zap.Error(err)) - return cerror.WrapError(cerror.ErrMySQLTxnError, err) + return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "create sync table: begin Tx fail;")) } _, err = tx.Exec("CREATE DATABASE IF NOT EXISTS " + database) if err != nil { @@ -161,7 +164,7 @@ func (s *mysqlSyncPointStore) CreateSyncTable(ctx context.Context) error { if err2 != nil { log.Error("failed to create syncpoint table", zap.Error(err2)) } - return cerror.WrapError(cerror.ErrMySQLTxnError, err) + return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to create syncpoint table;")) } _, err = tx.Exec("USE " + database) if err != nil { @@ -169,7 +172,7 @@ func (s *mysqlSyncPointStore) CreateSyncTable(ctx context.Context) error { if err2 != nil { log.Error("failed to create syncpoint table", zap.Error(err2)) } - return cerror.WrapError(cerror.ErrMySQLTxnError, err) + return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to create syncpoint table;")) } query := `CREATE TABLE IF NOT EXISTS %s ( @@ -188,10 +191,10 @@ func (s *mysqlSyncPointStore) CreateSyncTable(ctx context.Context) error { if err2 != nil { log.Error("failed to create syncpoint table", zap.Error(err2)) } - return cerror.WrapError(cerror.ErrMySQLTxnError, err) + return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to create syncpoint table;")) } err = tx.Commit() - return cerror.WrapError(cerror.ErrMySQLTxnError, err) + return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to create syncpoint table;")) } func (s *mysqlSyncPointStore) SinkSyncPoint(ctx context.Context, @@ -201,7 +204,7 @@ func (s *mysqlSyncPointStore) SinkSyncPoint(ctx context.Context, tx, err := s.db.BeginTx(ctx, nil) if err != nil { log.Error("sync table: begin Tx fail", zap.Error(err)) - return cerror.WrapError(cerror.ErrMySQLTxnError, err) + return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "sync table: begin Tx fail;")) } row := tx.QueryRow("select @@tidb_current_ts") var secondaryTs string @@ -212,7 +215,7 @@ func (s *mysqlSyncPointStore) SinkSyncPoint(ctx context.Context, if err2 != nil { log.Error("failed to write syncpoint table", zap.Error(err)) } - return cerror.WrapError(cerror.ErrMySQLTxnError, err) + return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to write syncpoint table;")) } // insert ts map query := "insert ignore into " + filter.TiCDCSystemSchema + "." + filter.SyncPointTable + @@ -223,7 +226,7 @@ func (s *mysqlSyncPointStore) SinkSyncPoint(ctx context.Context, if err2 != nil { log.Error("failed to write syncpoint table", zap.Error(err2)) } - return cerror.WrapError(cerror.ErrMySQLTxnError, err) + return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to write syncpoint table;")) } // set global tidb_external_ts to secondary ts @@ -239,7 +242,7 @@ func (s *mysqlSyncPointStore) SinkSyncPoint(ctx context.Context, if err2 != nil { log.Error("failed to write syncpoint table", zap.Error(err2)) } - return cerror.WrapError(cerror.ErrMySQLTxnError, err) + return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to write syncpoint table;")) } } @@ -264,7 +267,7 @@ func (s *mysqlSyncPointStore) SinkSyncPoint(ctx context.Context, } err = tx.Commit() - return cerror.WrapError(cerror.ErrMySQLTxnError, err) + return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to write syncpoint table;")) } func (s *mysqlSyncPointStore) Close() error { diff --git a/docs/swagger/docs.go b/docs/swagger/docs.go index 44d1f30f616..4277fb07f11 100644 --- a/docs/swagger/docs.go +++ b/docs/swagger/docs.go @@ -951,6 +951,57 @@ var doc = `{ } } }, + "/api/v2/changefeeds/{changefeed_id}/synced": { + "get": { + "description": "get the synced status of a changefeed", + "consumes": [ + "application/json" + ], + "produces": [ + "application/json" + ], + "tags": [ + "changefeed", + "v2" + ], + "summary": "Get synced status", + "parameters": [ + { + "type": "string", + "description": "changefeed_id", + "name": "changefeed_id", + "in": "path", + "required": true + }, + { + "type": "string", + "description": "default", + "name": "namespace", + "in": "query" + } + ], + "responses": { + "200": { + "description": "OK", + "schema": { + "$ref": "#/definitions/v2.SyncedStatus" + } + }, + "400": { + "description": "Bad Request", + "schema": { + "$ref": "#/definitions/model.HTTPError" + } + }, + "500": { + "description": "Internal Server Error", + "schema": { + "$ref": "#/definitions/model.HTTPError" + } + } + } + } + }, "/api/v2/health": { "get": { "description": "Check the health status of a TiCDC cluster", @@ -1209,6 +1260,36 @@ var doc = `{ } } }, + "config.CloudStorageConfig": { + "type": "object", + "properties": { + "file-cleanup-cron-spec": { + "type": "string" + }, + "file-expiration-days": { + "type": "integer" + }, + "file-size": { + "type": "integer" + }, + "flush-concurrency": { + "type": "integer" + }, + "flush-interval": { + "type": "string" + }, + "output-column-id": { + "type": "boolean" + }, + "output-raw-change-event": { + "description": "OutputRawChangeEvent controls whether to split the update pk/uk events.", + "type": "boolean" + }, + "worker-count": { + "type": "integer" + } + } + }, "config.ColumnSelector": { "type": "object", "properties": { @@ -1254,6 +1335,10 @@ var doc = `{ "large-message-handle": { "$ref": "#/definitions/config.LargeMessageHandleConfig" }, + "output-raw-change-event": { + "description": "OutputRawChangeEvent controls whether to split the update pk/uk events.", + "type": "boolean" + }, "sasl-mechanism": { "type": "string" }, @@ -1295,6 +1380,9 @@ var doc = `{ "description": "AdvanceTimeoutInSec is a duration in second. If a table sink progress hasn't been\nadvanced for this given duration, the sink will be canceled and re-established.", "type": "integer" }, + "cloud-storage-config": { + "$ref": "#/definitions/config.CloudStorageConfig" + }, "column-selectors": { "type": "array", "items": { @@ -1308,6 +1396,7 @@ var doc = `{ "type": "string" }, "dispatchers": { + "description": "DispatchRules is only available when the downstream is MQ.", "type": "array", "items": { "$ref": "#/definitions/config.DispatchRule" @@ -1735,6 +1824,35 @@ var doc = `{ } } }, + "v2.CloudStorageConfig": { + "type": "object", + "properties": { + "file_cleanup_cron_spec": { + "type": "string" + }, + "file_expiration_days": { + "type": "integer" + }, + "file_size": { + "type": "integer" + }, + "flush_concurrency": { + "type": "integer" + }, + "flush_interval": { + "type": "string" + }, + "output_column_id": { + "type": "boolean" + }, + "output_raw_change_event": { + "type": "boolean" + }, + "worker_count": { + "type": "integer" + } + } + }, "v2.ColumnSelector": { "type": "object", "properties": { @@ -1755,9 +1873,15 @@ var doc = `{ "v2.ConsistentConfig": { "type": "object", "properties": { + "compression": { + "type": "string" + }, "encoding_worker_num": { "type": "integer" }, + "flush_concurrency": { + "type": "integer" + }, "flush_interval": { "type": "integer" }, @@ -1770,6 +1894,9 @@ var doc = `{ "max_log_size": { "type": "integer" }, + "memory_usage": { + "$ref": "#/definitions/v2.ConsistentMemoryUsage" + }, "meta_flush_interval": { "type": "integer" }, @@ -1781,6 +1908,17 @@ var doc = `{ } } }, + "v2.ConsistentMemoryUsage": { + "type": "object", + "properties": { + "event_cache_percentage": { + "type": "integer" + }, + "memory_quota_percentage": { + "type": "integer" + } + } + }, "v2.DispatchRule": { "type": "object", "properties": { @@ -1895,6 +2033,9 @@ var doc = `{ "large_message_handle": { "$ref": "#/definitions/v2.LargeMessageHandleConfig" }, + "output_raw_change_event": { + "type": "boolean" + }, "sasl_mechanism": { "type": "string" }, @@ -1980,6 +2121,9 @@ var doc = `{ "case_sensitive": { "type": "boolean" }, + "changefeed_error_stuck_duration": { + "type": "string" + }, "check_gc_safe_point": { "type": "boolean" }, @@ -1992,6 +2136,9 @@ var doc = `{ "enable_sync_point": { "type": "boolean" }, + "enable_table_monitor": { + "type": "boolean" + }, "filter": { "$ref": "#/definitions/v2.FilterConfig" }, @@ -2010,11 +2157,17 @@ var doc = `{ "sink": { "$ref": "#/definitions/v2.SinkConfig" }, + "sql_mode": { + "type": "string" + }, "sync_point_interval": { "type": "string" }, "sync_point_retention": { "type": "string" + }, + "synced_status": { + "$ref": "#/definitions/v2.SyncedStatusConfig" } } }, @@ -2067,6 +2220,9 @@ var doc = `{ "advance_timeout": { "type": "integer" }, + "cloud_storage_config": { + "$ref": "#/definitions/v2.CloudStorageConfig" + }, "column_selectors": { "type": "array", "items": { @@ -2111,6 +2267,42 @@ var doc = `{ } } }, + "v2.SyncedStatus": { + "type": "object", + "properties": { + "info": { + "type": "string" + }, + "last_synced_ts": { + "type": "string" + }, + "now_ts": { + "type": "string" + }, + "puller_resolved_ts": { + "type": "string" + }, + "sink_checkpoint_ts": { + "type": "string" + }, + "synced": { + "type": "boolean" + } + } + }, + "v2.SyncedStatusConfig": { + "type": "object", + "properties": { + "checkpoint_interval": { + "description": "The maximum interval between latest checkpoint ts and now or\nbetween latest sink's checkpoint ts and puller's checkpoint ts required to reach synced state", + "type": "integer" + }, + "synced_check_interval": { + "description": "The minimum interval between the latest synced ts and now required to reach synced state", + "type": "integer" + } + } + }, "v2.Table": { "type": "object", "properties": { diff --git a/docs/swagger/swagger.json b/docs/swagger/swagger.json index acb93660a2b..f3a47b29781 100644 --- a/docs/swagger/swagger.json +++ b/docs/swagger/swagger.json @@ -932,6 +932,57 @@ } } }, + "/api/v2/changefeeds/{changefeed_id}/synced": { + "get": { + "description": "get the synced status of a changefeed", + "consumes": [ + "application/json" + ], + "produces": [ + "application/json" + ], + "tags": [ + "changefeed", + "v2" + ], + "summary": "Get synced status", + "parameters": [ + { + "type": "string", + "description": "changefeed_id", + "name": "changefeed_id", + "in": "path", + "required": true + }, + { + "type": "string", + "description": "default", + "name": "namespace", + "in": "query" + } + ], + "responses": { + "200": { + "description": "OK", + "schema": { + "$ref": "#/definitions/v2.SyncedStatus" + } + }, + "400": { + "description": "Bad Request", + "schema": { + "$ref": "#/definitions/model.HTTPError" + } + }, + "500": { + "description": "Internal Server Error", + "schema": { + "$ref": "#/definitions/model.HTTPError" + } + } + } + } + }, "/api/v2/health": { "get": { "description": "Check the health status of a TiCDC cluster", @@ -1190,6 +1241,36 @@ } } }, + "config.CloudStorageConfig": { + "type": "object", + "properties": { + "file-cleanup-cron-spec": { + "type": "string" + }, + "file-expiration-days": { + "type": "integer" + }, + "file-size": { + "type": "integer" + }, + "flush-concurrency": { + "type": "integer" + }, + "flush-interval": { + "type": "string" + }, + "output-column-id": { + "type": "boolean" + }, + "output-raw-change-event": { + "description": "OutputRawChangeEvent controls whether to split the update pk/uk events.", + "type": "boolean" + }, + "worker-count": { + "type": "integer" + } + } + }, "config.ColumnSelector": { "type": "object", "properties": { @@ -1235,6 +1316,10 @@ "large-message-handle": { "$ref": "#/definitions/config.LargeMessageHandleConfig" }, + "output-raw-change-event": { + "description": "OutputRawChangeEvent controls whether to split the update pk/uk events.", + "type": "boolean" + }, "sasl-mechanism": { "type": "string" }, @@ -1276,6 +1361,9 @@ "description": "AdvanceTimeoutInSec is a duration in second. If a table sink progress hasn't been\nadvanced for this given duration, the sink will be canceled and re-established.", "type": "integer" }, + "cloud-storage-config": { + "$ref": "#/definitions/config.CloudStorageConfig" + }, "column-selectors": { "type": "array", "items": { @@ -1289,6 +1377,7 @@ "type": "string" }, "dispatchers": { + "description": "DispatchRules is only available when the downstream is MQ.", "type": "array", "items": { "$ref": "#/definitions/config.DispatchRule" @@ -1716,6 +1805,35 @@ } } }, + "v2.CloudStorageConfig": { + "type": "object", + "properties": { + "file_cleanup_cron_spec": { + "type": "string" + }, + "file_expiration_days": { + "type": "integer" + }, + "file_size": { + "type": "integer" + }, + "flush_concurrency": { + "type": "integer" + }, + "flush_interval": { + "type": "string" + }, + "output_column_id": { + "type": "boolean" + }, + "output_raw_change_event": { + "type": "boolean" + }, + "worker_count": { + "type": "integer" + } + } + }, "v2.ColumnSelector": { "type": "object", "properties": { @@ -1736,9 +1854,15 @@ "v2.ConsistentConfig": { "type": "object", "properties": { + "compression": { + "type": "string" + }, "encoding_worker_num": { "type": "integer" }, + "flush_concurrency": { + "type": "integer" + }, "flush_interval": { "type": "integer" }, @@ -1751,6 +1875,9 @@ "max_log_size": { "type": "integer" }, + "memory_usage": { + "$ref": "#/definitions/v2.ConsistentMemoryUsage" + }, "meta_flush_interval": { "type": "integer" }, @@ -1762,6 +1889,17 @@ } } }, + "v2.ConsistentMemoryUsage": { + "type": "object", + "properties": { + "event_cache_percentage": { + "type": "integer" + }, + "memory_quota_percentage": { + "type": "integer" + } + } + }, "v2.DispatchRule": { "type": "object", "properties": { @@ -1876,6 +2014,9 @@ "large_message_handle": { "$ref": "#/definitions/v2.LargeMessageHandleConfig" }, + "output_raw_change_event": { + "type": "boolean" + }, "sasl_mechanism": { "type": "string" }, @@ -1961,6 +2102,9 @@ "case_sensitive": { "type": "boolean" }, + "changefeed_error_stuck_duration": { + "type": "string" + }, "check_gc_safe_point": { "type": "boolean" }, @@ -1973,6 +2117,9 @@ "enable_sync_point": { "type": "boolean" }, + "enable_table_monitor": { + "type": "boolean" + }, "filter": { "$ref": "#/definitions/v2.FilterConfig" }, @@ -1991,11 +2138,17 @@ "sink": { "$ref": "#/definitions/v2.SinkConfig" }, + "sql_mode": { + "type": "string" + }, "sync_point_interval": { "type": "string" }, "sync_point_retention": { "type": "string" + }, + "synced_status": { + "$ref": "#/definitions/v2.SyncedStatusConfig" } } }, @@ -2048,6 +2201,9 @@ "advance_timeout": { "type": "integer" }, + "cloud_storage_config": { + "$ref": "#/definitions/v2.CloudStorageConfig" + }, "column_selectors": { "type": "array", "items": { @@ -2092,6 +2248,42 @@ } } }, + "v2.SyncedStatus": { + "type": "object", + "properties": { + "info": { + "type": "string" + }, + "last_synced_ts": { + "type": "string" + }, + "now_ts": { + "type": "string" + }, + "puller_resolved_ts": { + "type": "string" + }, + "sink_checkpoint_ts": { + "type": "string" + }, + "synced": { + "type": "boolean" + } + } + }, + "v2.SyncedStatusConfig": { + "type": "object", + "properties": { + "checkpoint_interval": { + "description": "The maximum interval between latest checkpoint ts and now or\nbetween latest sink's checkpoint ts and puller's checkpoint ts required to reach synced state", + "type": "integer" + }, + "synced_check_interval": { + "description": "The minimum interval between the latest synced ts and now required to reach synced state", + "type": "integer" + } + } + }, "v2.Table": { "type": "object", "properties": { diff --git a/docs/swagger/swagger.yaml b/docs/swagger/swagger.yaml index 3820d2165d5..3716bac40dc 100644 --- a/docs/swagger/swagger.yaml +++ b/docs/swagger/swagger.yaml @@ -17,6 +17,27 @@ definitions: description: quoting character type: string type: object + config.CloudStorageConfig: + properties: + file-cleanup-cron-spec: + type: string + file-expiration-days: + type: integer + file-size: + type: integer + flush-concurrency: + type: integer + flush-interval: + type: string + output-column-id: + type: boolean + output-raw-change-event: + description: OutputRawChangeEvent controls whether to split the update pk/uk + events. + type: boolean + worker-count: + type: integer + type: object config.ColumnSelector: properties: columns: @@ -49,6 +70,10 @@ definitions: properties: large-message-handle: $ref: '#/definitions/config.LargeMessageHandleConfig' + output-raw-change-event: + description: OutputRawChangeEvent controls whether to split the update pk/uk + events. + type: boolean sasl-mechanism: type: string sasl-oauth-audience: @@ -78,6 +103,8 @@ definitions: AdvanceTimeoutInSec is a duration in second. If a table sink progress hasn't been advanced for this given duration, the sink will be canceled and re-established. type: integer + cloud-storage-config: + $ref: '#/definitions/config.CloudStorageConfig' column-selectors: items: $ref: '#/definitions/config.ColumnSelector' @@ -87,6 +114,7 @@ definitions: date-separator: type: string dispatchers: + description: DispatchRules is only available when the downstream is MQ. items: $ref: '#/definitions/config.DispatchRule' type: array @@ -373,6 +401,25 @@ definitions: upstream_id: type: integer type: object + v2.CloudStorageConfig: + properties: + file_cleanup_cron_spec: + type: string + file_expiration_days: + type: integer + file_size: + type: integer + flush_concurrency: + type: integer + flush_interval: + type: string + output_column_id: + type: boolean + output_raw_change_event: + type: boolean + worker_count: + type: integer + type: object v2.ColumnSelector: properties: columns: @@ -386,8 +433,12 @@ definitions: type: object v2.ConsistentConfig: properties: + compression: + type: string encoding_worker_num: type: integer + flush_concurrency: + type: integer flush_interval: type: integer flush_worker_num: @@ -396,6 +447,8 @@ definitions: type: string max_log_size: type: integer + memory_usage: + $ref: '#/definitions/v2.ConsistentMemoryUsage' meta_flush_interval: type: integer storage: @@ -403,6 +456,13 @@ definitions: use_file_backend: type: boolean type: object + v2.ConsistentMemoryUsage: + properties: + event_cache_percentage: + type: integer + memory_quota_percentage: + type: integer + type: object v2.DispatchRule: properties: matcher: @@ -480,6 +540,8 @@ definitions: properties: large_message_handle: $ref: '#/definitions/v2.LargeMessageHandleConfig' + output_raw_change_event: + type: boolean sasl_mechanism: type: string sasl_oauth_audience: @@ -535,6 +597,8 @@ definitions: type: boolean case_sensitive: type: boolean + changefeed_error_stuck_duration: + type: string check_gc_safe_point: type: boolean consistent: @@ -543,6 +607,8 @@ definitions: type: boolean enable_sync_point: type: boolean + enable_table_monitor: + type: boolean filter: $ref: '#/definitions/v2.FilterConfig' force_replicate: @@ -555,10 +621,14 @@ definitions: $ref: '#/definitions/v2.MounterConfig' sink: $ref: '#/definitions/v2.SinkConfig' + sql_mode: + type: string sync_point_interval: type: string sync_point_retention: type: string + synced_status: + $ref: '#/definitions/v2.SyncedStatusConfig' type: object v2.RunningError: properties: @@ -592,6 +662,8 @@ definitions: properties: advance_timeout: type: integer + cloud_storage_config: + $ref: '#/definitions/v2.CloudStorageConfig' column_selectors: items: $ref: '#/definitions/v2.ColumnSelector' @@ -621,6 +693,33 @@ definitions: transaction_atomicity: type: string type: object + v2.SyncedStatus: + properties: + info: + type: string + last_synced_ts: + type: string + now_ts: + type: string + puller_resolved_ts: + type: string + sink_checkpoint_ts: + type: string + synced: + type: boolean + type: object + v2.SyncedStatusConfig: + properties: + checkpoint_interval: + description: |- + The maximum interval between latest checkpoint ts and now or + between latest sink's checkpoint ts and puller's checkpoint ts required to reach synced state + type: integer + synced_check_interval: + description: The minimum interval between the latest synced ts and now required + to reach synced state + type: integer + type: object v2.Table: properties: database_name: @@ -1244,6 +1343,40 @@ paths: tags: - changefeed - v2 + /api/v2/changefeeds/{changefeed_id}/synced: + get: + consumes: + - application/json + description: get the synced status of a changefeed + parameters: + - description: changefeed_id + in: path + name: changefeed_id + required: true + type: string + - description: default + in: query + name: namespace + type: string + produces: + - application/json + responses: + "200": + description: OK + schema: + $ref: '#/definitions/v2.SyncedStatus' + "400": + description: Bad Request + schema: + $ref: '#/definitions/model.HTTPError' + "500": + description: Internal Server Error + schema: + $ref: '#/definitions/model.HTTPError' + summary: Get synced status + tags: + - changefeed + - v2 /api/v2/health: get: description: Check the health status of a TiCDC cluster diff --git a/errors.toml b/errors.toml index b386a96bf8a..996e76f22bc 100755 --- a/errors.toml +++ b/errors.toml @@ -1071,6 +1071,11 @@ error = ''' service safepoint lost. current safepoint is %d, please remove all changefeed(s) whose checkpoints are behind the current safepoint ''' +["CDC:ErrSinkIncompatibleConfig"] +error = ''' +incompatible configuration %s +''' + ["CDC:ErrSinkInvalidConfig"] error = ''' sink config invalid diff --git a/go.mod b/go.mod index 90bfaeed7ef..0d0ae184a2d 100644 --- a/go.mod +++ b/go.mod @@ -71,7 +71,6 @@ require ( github.com/r3labs/diff v1.1.0 github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 github.com/robfig/cron v1.2.0 - github.com/shirou/gopsutil/v3 v3.23.5 github.com/shopspring/decimal v1.3.0 github.com/soheilhy/cmux v0.1.5 github.com/spf13/cobra v1.6.1 @@ -247,6 +246,7 @@ require ( github.com/rogpeppe/go-internal v1.10.0 // indirect github.com/rs/cors v1.7.0 // indirect github.com/segmentio/asm v1.2.0 // indirect + github.com/shirou/gopsutil/v3 v3.23.5 // indirect github.com/shoenig/go-m1cpu v0.1.6 // indirect github.com/shurcooL/httpfs v0.0.0-20190707220628-8d4bc4ba7749 // indirect github.com/shurcooL/httpgzip v0.0.0-20190720172056-320755c1c1b0 // indirect diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index 1406df39049..e27d56f8151 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -7481,9 +7481,9 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(ticdc_sinkv2_txn_worker_busy_ratio{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])/10) by (changefeed,instance)", + "expr": "sum(rate(ticdc_sinkv2_txn_worker_flush_duration_sum{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",namespace=~\"$namespace\",changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (namespace,changefeed,instance,id) /sum(rate(ticdc_sink_txn_worker_total_duration_sum{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",namespace=~\"$namespace\",changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (namespace,changefeed,instance,id) *100", "interval": "", - "legendFormat": "{{changefeed}}-{{instance}}", + "legendFormat": "{{namespace}}-{{changefeed}}-{{instance}}-worker-{{id}}", "queryType": "randomWalk", "refId": "A" } diff --git a/pkg/applier/redo.go b/pkg/applier/redo.go index b944c8ea258..7749fb000f9 100644 --- a/pkg/applier/redo.go +++ b/pkg/applier/redo.go @@ -574,10 +574,10 @@ func (t *tempTxnInsertEventStorage) readNextEvent() (*model.RowChangedEvent, err // when the update event is an update to the handle key or the non empty unique key. // deferred insert event means all delete events and update events in the same transaction are emitted before this insert event type updateEventSplitter struct { - rd reader.RedoLogReader - rdFinished bool - tempStorage *tempTxnInsertEventStorage - prevTxnCommitTs model.Ts + rd reader.RedoLogReader + rdFinished bool + tempStorage *tempTxnInsertEventStorage + prevTxnStartTs model.Ts // pendingEvent is the event that trigger the process to emit events from tempStorage, it can be // 1) an insert event in the same transaction(because there will be no more update and delete events in the same transaction) // 2) a new event in the next transaction @@ -589,17 +589,17 @@ type updateEventSplitter struct { func newUpdateEventSplitter(rd reader.RedoLogReader, dir string) *updateEventSplitter { return &updateEventSplitter{ - rd: rd, - rdFinished: false, - tempStorage: newTempTxnInsertEventStorage(defaultFlushThreshold, dir), - prevTxnCommitTs: 0, + rd: rd, + rdFinished: false, + tempStorage: newTempTxnInsertEventStorage(defaultFlushThreshold, dir), + prevTxnStartTs: 0, } } // processEvent return (event to emit, pending event) func processEvent( event *model.RowChangedEvent, - prevTxnCommitTs model.Ts, + prevTxnStartTs model.Ts, tempStorage *tempTxnInsertEventStorage, ) (*model.RowChangedEvent, *model.RowChangedEvent, error) { if event == nil { @@ -607,7 +607,7 @@ func processEvent( } // meet a new transaction - if prevTxnCommitTs != 0 && prevTxnCommitTs != event.CommitTs { + if prevTxnStartTs != 0 && prevTxnStartTs != event.StartTs { if tempStorage.hasEvent() { // emit the insert events in the previous transaction return nil, event, nil @@ -640,7 +640,8 @@ func (u *updateEventSplitter) checkEventOrder(event *model.RowChangedEvent) { if event == nil { return } - if event.CommitTs > u.prevTxnCommitTs { + // meeet a new transaction + if event.StartTs != u.prevTxnStartTs { u.meetInsertInCurTxn = false return } @@ -663,7 +664,7 @@ func (u *updateEventSplitter) readNextRow(ctx context.Context) (*model.RowChange } var event *model.RowChangedEvent var err error - event, u.pendingEvent, err = processEvent(u.pendingEvent, u.prevTxnCommitTs, u.tempStorage) + event, u.pendingEvent, err = processEvent(u.pendingEvent, u.prevTxnStartTs, u.tempStorage) if err != nil { return nil, err } @@ -690,10 +691,10 @@ func (u *updateEventSplitter) readNextRow(ctx context.Context) (*model.RowChange u.rdFinished = true } else { u.checkEventOrder(event) - prevTxnCommitTS := u.prevTxnCommitTs - u.prevTxnCommitTs = event.CommitTs + prevTxnStartTs := u.prevTxnStartTs + u.prevTxnStartTs = event.StartTs var err error - event, u.pendingEvent, err = processEvent(event, prevTxnCommitTS, u.tempStorage) + event, u.pendingEvent, err = processEvent(event, prevTxnStartTs, u.tempStorage) if err != nil { return nil, err } diff --git a/pkg/cmd/redo/apply.go b/pkg/cmd/redo/apply.go index b06276becc4..2e219f407da 100644 --- a/pkg/cmd/redo/apply.go +++ b/pkg/cmd/redo/apply.go @@ -14,18 +14,27 @@ package redo import ( + "net/http" + _ "net/http/pprof" //nolint:gosec "net/url" + "runtime/debug" + "time" + "github.com/pingcap/log" "github.com/pingcap/tiflow/pkg/applier" cmdcontext "github.com/pingcap/tiflow/pkg/cmd/context" cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/util" "github.com/spf13/cobra" + "go.uber.org/zap" ) // applyRedoOptions defines flags for the `redo apply` command. type applyRedoOptions struct { options - sinkURI string + sinkURI string + enableProfiling bool + memoryLimitInGiBytes int64 } // newapplyRedoOptions creates new applyRedoOptions for the `redo apply` command. @@ -39,6 +48,8 @@ func (o *applyRedoOptions) addFlags(cmd *cobra.Command) { cmd.Flags().StringVar(&o.sinkURI, "sink-uri", "", "target database sink-uri") // the possible error returned from MarkFlagRequired is `no such flag` cmd.MarkFlagRequired("sink-uri") //nolint:errcheck + cmd.Flags().BoolVar(&o.enableProfiling, "enable-profiling", true, "enable pprof profiling") + cmd.Flags().Int64Var(&o.memoryLimitInGiBytes, "memory-limit", 10, "memory limit in GiB") } //nolint:unparam @@ -55,6 +66,18 @@ func (o *applyRedoOptions) complete(cmd *cobra.Command) error { sinkURI.RawQuery = rawQuery.Encode() o.sinkURI = sinkURI.String() } + + totalMemory, err := util.GetMemoryLimit() + if err == nil { + totalMemoryInBytes := int64(float64(totalMemory) * 0.8) + memoryLimitInBytes := o.memoryLimitInGiBytes * 1024 * 1024 * 1024 + if totalMemoryInBytes != 0 && memoryLimitInBytes > totalMemoryInBytes { + memoryLimitInBytes = totalMemoryInBytes + } + debug.SetMemoryLimit(memoryLimitInBytes) + log.Info("set memory limit", zap.Int64("memoryLimit", memoryLimitInBytes)) + } + return nil } @@ -62,6 +85,19 @@ func (o *applyRedoOptions) complete(cmd *cobra.Command) error { func (o *applyRedoOptions) run(cmd *cobra.Command) error { ctx := cmdcontext.GetDefaultContext() + if o.enableProfiling { + go func() { + server := &http.Server{ + Addr: ":6060", + ReadHeaderTimeout: 5 * time.Second, + } + log.Info("Start http pprof server", zap.String("addr", server.Addr)) + if err := server.ListenAndServe(); err != nil { + log.Fatal("http pprof", zap.Error(err)) + } + }() + } + cfg := &applier.RedoApplierConfig{ Storage: o.storage, SinkURI: o.sinkURI, diff --git a/pkg/config/sink.go b/pkg/config/sink.go index 9f73988dd32..bfcc7dc0a0b 100644 --- a/pkg/config/sink.go +++ b/pkg/config/sink.go @@ -121,6 +121,7 @@ type SinkConfig struct { TxnAtomicity AtomicityLevel `toml:"transaction-atomicity" json:"transaction-atomicity"` Protocol string `toml:"protocol" json:"protocol"` + // DispatchRules is only available when the downstream is MQ. DispatchRules []*DispatchRule `toml:"dispatchers" json:"dispatchers"` CSVConfig *CSVConfig `toml:"csv" json:"csv"` ColumnSelectors []*ColumnSelector `toml:"column-selectors" json:"column-selectors"` @@ -155,6 +156,16 @@ type KafkaConfig struct { SASLOAuthAudience *string `toml:"sasl-oauth-audience" json:"sasl-oauth-audience,omitempty"` LargeMessageHandle *LargeMessageHandleConfig `toml:"large-message-handle" json:"large-message-handle,omitempty"` + // OutputRawChangeEvent controls whether to split the update pk/uk events. + OutputRawChangeEvent *bool `toml:"output-raw-change-event" json:"output-raw-change-event,omitempty"` +} + +// GetOutputRawChangeEvent returns the value of OutputRawChangeEvent +func (k *KafkaConfig) GetOutputRawChangeEvent() bool { + if k == nil || k.OutputRawChangeEvent == nil { + return false + } + return *k.OutputRawChangeEvent } // MaskSensitiveData masks sensitive data in SinkConfig @@ -403,21 +414,41 @@ func (s *SinkConfig) validateAndAdjustSinkURI(sinkURI *url.URL) error { return err } - // Validate 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(s.Protocol) - if err != nil { - return err - } - } else if sink.IsMySQLCompatibleScheme(sinkURI.Scheme) && s.Protocol != "" { + log.Info("succeed to parse parameter from sink uri", + zap.String("protocol", s.Protocol), + zap.String("txnAtomicity", string(s.TxnAtomicity))) + + // Check that protocol config is compatible with the scheme. + if sink.IsMySQLCompatibleScheme(sinkURI.Scheme) && s.Protocol != "" { return cerror.ErrSinkURIInvalid.GenWithStackByArgs(fmt.Sprintf("protocol %s "+ "is incompatible with %s scheme", s.Protocol, sinkURI.Scheme)) } + // For testing purposes, any protocol should be legal for blackhole. + if sink.IsMQScheme(sinkURI.Scheme) || sink.IsStorageScheme(sinkURI.Scheme) { + return s.ValidateProtocol(sinkURI.Scheme) + } + return nil +} - log.Info("succeed to parse parameter from sink uri", - zap.String("protocol", s.Protocol), - zap.String("txnAtomicity", string(s.TxnAtomicity))) +// ValidateProtocol validates the protocol configuration. +func (s *SinkConfig) ValidateProtocol(scheme string) error { + protocol, err := ParseSinkProtocolFromString(s.Protocol) + if err != nil { + return err + } + + outputRawChangeEvent := false + switch scheme { + case sink.KafkaScheme, sink.KafkaSSLScheme: + outputRawChangeEvent = s.KafkaConfig.GetOutputRawChangeEvent() + default: + outputRawChangeEvent = s.CloudStorageConfig.GetOutputRawChangeEvent() + } + + if outputRawChangeEvent { + // TODO: return error if we do not need to keep backward compatibility. + log.Warn(fmt.Sprintf("TiCDC will not split the update pk/uk events if output-raw-change-event is true(scheme: %s, protocol: %s).", scheme, protocol)) + } return nil } @@ -570,4 +601,15 @@ type CloudStorageConfig struct { OutputColumnID *bool `toml:"output-column-id" json:"output-column-id,omitempty"` FileExpirationDays *int `toml:"file-expiration-days" json:"file-expiration-days,omitempty"` FileCleanupCronSpec *string `toml:"file-cleanup-cron-spec" json:"file-cleanup-cron-spec,omitempty"` + + // OutputRawChangeEvent controls whether to split the update pk/uk events. + OutputRawChangeEvent *bool `toml:"output-raw-change-event" json:"output-raw-change-event,omitempty"` +} + +// GetOutputRawChangeEvent returns the value of OutputRawChangeEvent +func (c *CloudStorageConfig) GetOutputRawChangeEvent() bool { + if c == nil || c.OutputRawChangeEvent == nil { + return false + } + return *c.OutputRawChangeEvent } diff --git a/pkg/errors/cdc_errors.go b/pkg/errors/cdc_errors.go index 3a633f2da68..dbd25e03813 100644 --- a/pkg/errors/cdc_errors.go +++ b/pkg/errors/cdc_errors.go @@ -412,6 +412,10 @@ var ( "sink config invalid", errors.RFCCodeText("CDC:ErrSinkInvalidConfig"), ) + ErrSinkIncompatibleConfig = errors.Normalize( + "incompatible configuration %s", + errors.RFCCodeText("CDC:ErrSinkIncompatibleConfig"), + ) ErrCraftCodecInvalidData = errors.Normalize( "craft codec invalid data", errors.RFCCodeText("CDC:ErrCraftCodecInvalidData"), diff --git a/pkg/sink/mysql/db_helper.go b/pkg/sink/mysql/db_helper.go index 59be0339100..decad51fc9e 100644 --- a/pkg/sink/mysql/db_helper.go +++ b/pkg/sink/mysql/db_helper.go @@ -318,12 +318,7 @@ func CheckIsTiDB(ctx context.Context, db *sql.DB) (bool, error) { err := row.Scan(&tidbVer) if err != nil { log.Error("check tidb version error", zap.Error(err)) - // downstream is not TiDB, do nothing - if mysqlErr, ok := errors.Cause(err).(*dmysql.MySQLError); ok && (mysqlErr.Number == tmysql.ErrNoDB || - mysqlErr.Number == tmysql.ErrSpDoesNotExist || mysqlErr.Number == tmysql.ErrDBaccessDenied) { - return false, nil - } - return false, errors.Trace(err) + return false, nil } return true, nil } diff --git a/pkg/util/memory_checker.go b/pkg/util/memory_checker.go deleted file mode 100644 index 8132edadc85..00000000000 --- a/pkg/util/memory_checker.go +++ /dev/null @@ -1,47 +0,0 @@ -// 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 ( - "time" - - "github.com/pingcap/tiflow/pkg/errors" - "github.com/shirou/gopsutil/v3/mem" -) - -// CheckMemoryUsage checks if the memory usage is less than the limit. -func CheckMemoryUsage(limit float64) (bool, error) { - stat, err := mem.VirtualMemory() - if err != nil { - return false, err - } - return stat.UsedPercent < limit, nil -} - -// WaitMemoryAvailable waits until the memory usage is less than the limit. -func WaitMemoryAvailable(limit float64, timeout time.Duration) error { - start := time.Now() - for { - hasFreeMemory, err := CheckMemoryUsage(limit) - if err != nil { - return err - } - if hasFreeMemory { - return nil - } - if time.Since(start) > timeout { - return errors.ErrWaitFreeMemoryTimeout.GenWithStackByArgs() - } - } -} diff --git a/scripts/download-integration-test-binaries.sh b/scripts/download-integration-test-binaries.sh index 4c4e9ef1e69..34a165c913d 100755 --- a/scripts/download-integration-test-binaries.sh +++ b/scripts/download-integration-test-binaries.sh @@ -48,7 +48,7 @@ tiflash_download_url="${file_server_url}/download/builds/pingcap/tiflash/${branc minio_download_url="${file_server_url}/download/minio.tar.gz" go_ycsb_download_url="${file_server_url}/download/builds/pingcap/go-ycsb/test-br/go-ycsb" etcd_download_url="${file_server_url}/download/builds/pingcap/cdc/etcd-v3.4.7-linux-amd64.tar.gz" -sync_diff_inspector_url="${file_server_url}/download/builds/pingcap/cdc/sync_diff_inspector_hash-00998a9a_linux-amd64.tar.gz" +sync_diff_inspector_url="${file_server_url}/download/builds/pingcap/cdc/sync_diff_inspector_hash-79f1fd1e_linux-amd64.tar.gz" jq_download_url="${file_server_url}/download/builds/pingcap/test/jq-1.6/jq-linux64" # Some temporary dir. diff --git a/tests/integration_tests/bank/run.sh b/tests/integration_tests/bank/run.sh index d883e16dc01..af38de33720 100644 --- a/tests/integration_tests/bank/run.sh +++ b/tests/integration_tests/bank/run.sh @@ -31,14 +31,15 @@ function prepare() { trap stop_tidb_cluster EXIT # kafka is not supported yet. if [ "$SINK_TYPE" != "kafka" ]; then - prepare $* + # TODO: enable bank test after it is stable enough. + # prepare $* - cd "$(dirname "$0")" - set -euxo pipefail + # cd "$(dirname "$0")" + # set -euxo pipefail - GO111MODULE=on go run bank.go case.go -u "root@tcp(${UP_TIDB_HOST}:${UP_TIDB_PORT})/bank" \ - -d "root@tcp(${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT})/bank" --test-round=20000 + # GO111MODULE=on go run bank.go case.go -u "root@tcp(${UP_TIDB_HOST}:${UP_TIDB_PORT})/bank" \ + # -d "root@tcp(${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT})/bank" --test-round=20000 - cleanup_process $CDC_BINARY + # cleanup_process $CDC_BINARY echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" fi diff --git a/tests/integration_tests/csv_storage_update_pk_clustered/conf/changefeed1.toml b/tests/integration_tests/csv_storage_update_pk_clustered/conf/changefeed1.toml new file mode 100644 index 00000000000..67f590af9bb --- /dev/null +++ b/tests/integration_tests/csv_storage_update_pk_clustered/conf/changefeed1.toml @@ -0,0 +1,25 @@ +# Case 1: default configuration where `csv.output-old-value=false` and `sink.cloud-storage-config.output-raw-change-event=false` +# Split and sort update pk/uk events in table sink. + +[filter] +rules = ['test.*'] + +[sink] +protocol = "csv" +# Line terminator. Empty value means "\r\n" (CRLF) is line terminators. The default value is empty. +terminator = "\n" +# Directory date separator, Optional values are `none`, `year`, `month`, `date`. The default value is none. +date-separator = 'day' + +[sink.cloud-storage-config] +output-raw-change-event = false + +[sink.csv] +# Delimiter between fields. Must be ASCII characters. The default value is ','. +delimiter = ',' +# Quoting character. Empty value means no quoting. The default value is '"'. +quote = '"' +# Representation of null values in CSV files, the default value is '\N' +null = '\N' +# Include commit-ts in the row data. The default value is false. +include-commit-ts = true \ No newline at end of file diff --git a/tests/integration_tests/csv_storage_update_pk_clustered/conf/changefeed2.toml b/tests/integration_tests/csv_storage_update_pk_clustered/conf/changefeed2.toml new file mode 100644 index 00000000000..b806252b395 --- /dev/null +++ b/tests/integration_tests/csv_storage_update_pk_clustered/conf/changefeed2.toml @@ -0,0 +1,24 @@ +# Case 2: Split all update events in csv encoder. + +[filter] +rules = ['test.*'] + +[sink] +protocol = "csv" +# Line terminator. Empty value means "\r\n" (CRLF) is line terminators. The default value is empty. +terminator = "\n" +# Directory date separator, Optional values are `none`, `year`, `month`, `date`. The default value is none. +date-separator = 'day' + +[sink.cloud-storage-config] +output-raw-change-event = true + +[sink.csv] +# Delimiter between fields. Must be ASCII characters. The default value is ','. +delimiter = ',' +# Quoting character. Empty value means no quoting. The default value is '"'. +quote = '"' +# Representation of null values in CSV files, the default value is '\N' +null = '\N' +# Include commit-ts in the row data. The default value is false. +include-commit-ts = true \ No newline at end of file diff --git a/tests/integration_tests/csv_storage_update_pk_clustered/conf/diff_config.toml b/tests/integration_tests/csv_storage_update_pk_clustered/conf/diff_config.toml new file mode 100644 index 00000000000..8edf2368fa4 --- /dev/null +++ b/tests/integration_tests/csv_storage_update_pk_clustered/conf/diff_config.toml @@ -0,0 +1,29 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] + output-dir = "/tmp/tidb_cdc_test/csv_storage_update_pk_clustered/sync_diff/output" + + source-instances = ["mysql1"] + + target-instance = "tidb0" + + target-check-tables = ["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/csv_storage_update_pk_clustered/data/prepare.sql b/tests/integration_tests/csv_storage_update_pk_clustered/data/prepare.sql new file mode 100644 index 00000000000..506a6e75765 --- /dev/null +++ b/tests/integration_tests/csv_storage_update_pk_clustered/data/prepare.sql @@ -0,0 +1,27 @@ +drop database if exists `test`; +create database `test`; +use `test`; + +CREATE TABLE `update_pk` ( + `id` int PRIMARY KEY CLUSTERED, + `pad` varchar(100) NOT NULL +); +INSERT INTO `update_pk` (`id`, `pad`) VALUES (1, 'example1'), (2, 'example2'); +INSERT INTO `update_pk` (`id`, `pad`) VALUES (10, 'example10'), (20, 'example20'); +INSERT INTO `update_pk` (`id`, `pad`) VALUES (100, 'example100'); +INSERT INTO `update_pk` (`id`, `pad`) VALUES (1000, 'example1000'); + +SHOW INDEX FROM update_pk; + +CREATE TABLE `update_uk` ( + `id` int PRIMARY KEY CLUSTERED, + `uk` int NOT NULL, + `pad` varchar(100) NOT NULL, + UNIQUE KEY `uk` (`uk`) +); +INSERT INTO `update_uk` (`id`, `uk`, `pad`) VALUES (1, 1, 'example1'), (2, 2, 'example2'); +INSERT INTO `update_uk` (`id`, `uk`, `pad`) VALUES (10, 10, 'example10'), (20, 20, 'example20'); +INSERT INTO `update_uk` (`id`, `uk`, `pad`) VALUES (100, 100, 'example100'); +INSERT INTO `update_uk` (`id`, `uk`, `pad`) VALUES (1000, 1000, 'example1000'); + +SHOW INDEX FROM update_uk; \ No newline at end of file diff --git a/tests/integration_tests/csv_storage_update_pk_clustered/data/run.sql b/tests/integration_tests/csv_storage_update_pk_clustered/data/run.sql new file mode 100644 index 00000000000..86e7d7e7d77 --- /dev/null +++ b/tests/integration_tests/csv_storage_update_pk_clustered/data/run.sql @@ -0,0 +1,40 @@ +USE `test`; + +-- update_pk -- + +BEGIN; -- Note: multi-row exchange +UPDATE update_pk SET id = 3 WHERE id = 1; +UPDATE update_pk SET id = 1 WHERE id = 2; +UPDATE update_pk SET id = 2 WHERE id = 3; +COMMIT; + +BEGIN; -- Note: multi-row update with no order dependency +UPDATE update_pk SET id = 30 WHERE id = 10; +UPDATE update_pk SET id = 40 WHERE id = 20; +COMMIT; + +BEGIN; -- Single row update +UPDATE update_pk SET id = 200 WHERE id = 100; +COMMIT; + +-- Normal update +UPDATE update_pk SET pad='example1001' WHERE id = 1000; + +-- update_uk -- +BEGIN; -- Note: multi-row exchange +UPDATE update_uk SET uk = 3 WHERE uk = 1; +UPDATE update_uk SET uk = 1 WHERE uk = 2; +UPDATE update_uk SET uk = 2 WHERE uk = 3; +COMMIT; + +BEGIN; -- Note: multi-row update with no order dependency +UPDATE update_uk SET uk = 30 WHERE uk = 10; +UPDATE update_uk SET uk = 40 WHERE uk = 20; +COMMIT; + +BEGIN; -- Single row update +UPDATE update_uk SET uk = 200 WHERE uk = 100; +COMMIT; + +-- Normal update +UPDATE update_uk SET pad='example1001' WHERE uk = 1000; \ No newline at end of file diff --git a/tests/integration_tests/csv_storage_update_pk_clustered/result/changefeed1_pk.res b/tests/integration_tests/csv_storage_update_pk_clustered/result/changefeed1_pk.res new file mode 100644 index 00000000000..ad6016c8059 --- /dev/null +++ b/tests/integration_tests/csv_storage_update_pk_clustered/result/changefeed1_pk.res @@ -0,0 +1,22 @@ +"I","update_pk","test",450253245302439944,1,"example1" +"I","update_pk","test",450253245302439944,2,"example2" +"I","update_pk","test",450253245302439946,10,"example10" +"I","update_pk","test",450253245302439946,20,"example20" +"I","update_pk","test",450253245302439947,100,"example100" +"I","update_pk","test",450253245302439948,1000,"example1000" + +# translate to normal update in upstream +"U","update_pk","test",450253245485940746,1,"example2" +"U","update_pk","test",450253245485940746,2,"example1" + +# split and sort in upstream +"D","update_pk","test",450253245485940749,10,"example10" +"D","update_pk","test",450253245485940749,20,"example20" +"I","update_pk","test",450253245485940749,30,"example10" +"I","update_pk","test",450253245485940749,40,"example20" + +# split and sort in upstream +"D","update_pk","test",450253245485940752,100,"example100" +"I","update_pk","test",450253245485940752,200,"example100" + +"U","update_pk","test",450253245485940753,1000,"example1001" diff --git a/tests/integration_tests/csv_storage_update_pk_clustered/result/changefeed1_uk.res b/tests/integration_tests/csv_storage_update_pk_clustered/result/changefeed1_uk.res new file mode 100644 index 00000000000..ebe3a635252 --- /dev/null +++ b/tests/integration_tests/csv_storage_update_pk_clustered/result/changefeed1_uk.res @@ -0,0 +1,24 @@ +"I","update_uk","test",450253245446619144,1,1,"example1" +"I","update_uk","test",450253245446619144,2,2,"example2" +"I","update_uk","test",450253245446619146,10,10,"example10" +"I","update_uk","test",450253245446619146,20,20,"example20" +"I","update_uk","test",450253245446619147,100,100,"example100" +"I","update_uk","test",450253245446619148,1000,1000,"example1000" + +# split and sort in table sink +"D","update_uk","test",450253245499047940,1,1,"example1" +"D","update_uk","test",450253245499047940,2,2,"example2" +"I","update_uk","test",450253245499047940,1,2,"example1" +"I","update_uk","test",450253245499047940,2,1,"example2" + +# split and sort in table sink +"D","update_uk","test",450253245499047943,10,10,"example10" +"D","update_uk","test",450253245499047943,20,20,"example20" +"I","update_uk","test",450253245499047943,10,30,"example10" +"I","update_uk","test",450253245499047943,20,40,"example20" + +# split and sort in table sink +"D","update_uk","test",450253245499047946,100,100,"example100" +"I","update_uk","test",450253245499047946,100,200,"example100" + +"U","update_uk","test",450253245512155140,1000,1000,"example1001" \ No newline at end of file diff --git a/tests/integration_tests/csv_storage_update_pk_clustered/result/changefeed2_pk.res b/tests/integration_tests/csv_storage_update_pk_clustered/result/changefeed2_pk.res new file mode 100644 index 00000000000..fc3ea45b65d --- /dev/null +++ b/tests/integration_tests/csv_storage_update_pk_clustered/result/changefeed2_pk.res @@ -0,0 +1,26 @@ +"I","update_pk","test",450253245302439944,false,1,"example1" +"I","update_pk","test",450253245302439944,false,2,"example2" +"I","update_pk","test",450253245302439946,false,10,"example10" +"I","update_pk","test",450253245302439946,false,20,"example20" +"I","update_pk","test",450253245302439947,false,100,"example100" +"I","update_pk","test",450253245302439948,false,1000,"example1000" + +# translate to normal update in upstream, split in csv encoder +"D","update_pk","test",450253245485940746,true,1,"example1" +"I","update_pk","test",450253245485940746,true,1,"example2" +"D","update_pk","test",450253245485940746,true,2,"example2" +"I","update_pk","test",450253245485940746,true,2,"example1" + +# split and sort in upstream +"D","update_pk","test",450253245485940749,false,10,"example10" +"D","update_pk","test",450253245485940749,false,20,"example20" +"I","update_pk","test",450253245485940749,false,30,"example10" +"I","update_pk","test",450253245485940749,false,40,"example20" + +# split and sort in upstream +"D","update_pk","test",450253245485940752,false,100,"example100" +"I","update_pk","test",450253245485940752,false,200,"example100" + +# normal update event, split in csv encoder +"D","update_pk","test",450253245485940753,true,1000,"example1000" +"I","update_pk","test",450253245485940753,true,1000,"example1001" diff --git a/tests/integration_tests/csv_storage_update_pk_clustered/result/changefeed2_uk.res b/tests/integration_tests/csv_storage_update_pk_clustered/result/changefeed2_uk.res new file mode 100644 index 00000000000..5e7f2ce0e71 --- /dev/null +++ b/tests/integration_tests/csv_storage_update_pk_clustered/result/changefeed2_uk.res @@ -0,0 +1,26 @@ +"I","update_uk","test",450253245446619144,false,1,1,"example1" +"I","update_uk","test",450253245446619144,false,2,2,"example2" +"I","update_uk","test",450253245446619146,false,10,10,"example10" +"I","update_uk","test",450253245446619146,false,20,20,"example20" +"I","update_uk","test",450253245446619147,false,100,100,"example100" +"I","update_uk","test",450253245446619148,false,1000,1000,"example1000" + +# split in csv encoder, data is consistent since delete by pk +"D","update_uk","test",450253245499047940,true,1,1,"example1" +"I","update_uk","test",450253245499047940,true,1,2,"example1" +"D","update_uk","test",450253245499047940,true,2,2,"example2" +"I","update_uk","test",450253245499047940,true,2,1,"example2" + +# split in csv encoder +"D","update_uk","test",450253245499047943,true,10,10,"example10" +"I","update_uk","test",450253245499047943,true,10,30,"example10" +"D","update_uk","test",450253245499047943,true,20,20,"example20" +"I","update_uk","test",450253245499047943,true,20,40,"example20" + +# split in csv encoder +"D","update_uk","test",450253245499047946,true,100,100,"example100" +"I","update_uk","test",450253245499047946,true,100,200,"example100" + +# normal update event, also split in csv encoder +"D","update_uk","test",450253245512155140,true,1000,1000,"example1000" +"I","update_uk","test",450253245512155140,true,1000,1000,"example1001" \ No newline at end of file diff --git a/tests/integration_tests/csv_storage_update_pk_clustered/run.sh b/tests/integration_tests/csv_storage_update_pk_clustered/run.sh new file mode 100644 index 00000000000..65842e07025 --- /dev/null +++ b/tests/integration_tests/csv_storage_update_pk_clustered/run.sh @@ -0,0 +1,52 @@ +#!/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_changefeed() { + local changefeed_id=$1 + local start_ts=$2 + local expected_split_count=$3 + SINK_URI="file://$WORK_DIR/storage_test/$changefeed_id?flush-interval=5s" + run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --config=$CUR/conf/$changefeed_id.toml -c "$changefeed_id" + + run_storage_consumer $WORK_DIR $SINK_URI $CUR/conf/$changefeed_id.toml $changefeed_id + sleep 8 + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 100 + + real_split_count=$(grep "split update event" $WORK_DIR/cdc.log | wc -l) + if [[ $real_split_count -ne $expected_split_count ]]; then + echo "expected split count $expected_split_count, real split count $real_split_count" + exit 1 + fi + run_sql "drop database if exists test" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} +} + +function run() { + if [ "$SINK_TYPE" != "storage" ]; then + return + fi + + rm -rf $WORK_DIR && mkdir -p $WORK_DIR + start_tidb_cluster --workdir $WORK_DIR + cd $WORK_DIR + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + + start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) + + run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql_file $CUR/data/run.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} + + run_changefeed "changefeed1" $start_ts 5 + run_changefeed "changefeed2" $start_ts 5 +} + +trap stop_tidb_cluster EXIT +run $* +check_logs $WORK_DIR +echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" diff --git a/tests/integration_tests/csv_storage_update_pk_nonclustered/conf/changefeed1.toml b/tests/integration_tests/csv_storage_update_pk_nonclustered/conf/changefeed1.toml new file mode 100644 index 00000000000..67f590af9bb --- /dev/null +++ b/tests/integration_tests/csv_storage_update_pk_nonclustered/conf/changefeed1.toml @@ -0,0 +1,25 @@ +# Case 1: default configuration where `csv.output-old-value=false` and `sink.cloud-storage-config.output-raw-change-event=false` +# Split and sort update pk/uk events in table sink. + +[filter] +rules = ['test.*'] + +[sink] +protocol = "csv" +# Line terminator. Empty value means "\r\n" (CRLF) is line terminators. The default value is empty. +terminator = "\n" +# Directory date separator, Optional values are `none`, `year`, `month`, `date`. The default value is none. +date-separator = 'day' + +[sink.cloud-storage-config] +output-raw-change-event = false + +[sink.csv] +# Delimiter between fields. Must be ASCII characters. The default value is ','. +delimiter = ',' +# Quoting character. Empty value means no quoting. The default value is '"'. +quote = '"' +# Representation of null values in CSV files, the default value is '\N' +null = '\N' +# Include commit-ts in the row data. The default value is false. +include-commit-ts = true \ No newline at end of file diff --git a/tests/integration_tests/csv_storage_update_pk_nonclustered/conf/changefeed2.toml b/tests/integration_tests/csv_storage_update_pk_nonclustered/conf/changefeed2.toml new file mode 100644 index 00000000000..b806252b395 --- /dev/null +++ b/tests/integration_tests/csv_storage_update_pk_nonclustered/conf/changefeed2.toml @@ -0,0 +1,24 @@ +# Case 2: Split all update events in csv encoder. + +[filter] +rules = ['test.*'] + +[sink] +protocol = "csv" +# Line terminator. Empty value means "\r\n" (CRLF) is line terminators. The default value is empty. +terminator = "\n" +# Directory date separator, Optional values are `none`, `year`, `month`, `date`. The default value is none. +date-separator = 'day' + +[sink.cloud-storage-config] +output-raw-change-event = true + +[sink.csv] +# Delimiter between fields. Must be ASCII characters. The default value is ','. +delimiter = ',' +# Quoting character. Empty value means no quoting. The default value is '"'. +quote = '"' +# Representation of null values in CSV files, the default value is '\N' +null = '\N' +# Include commit-ts in the row data. The default value is false. +include-commit-ts = true \ No newline at end of file diff --git a/tests/integration_tests/csv_storage_update_pk_nonclustered/conf/diff_config.toml b/tests/integration_tests/csv_storage_update_pk_nonclustered/conf/diff_config.toml new file mode 100644 index 00000000000..0714c0c18d9 --- /dev/null +++ b/tests/integration_tests/csv_storage_update_pk_nonclustered/conf/diff_config.toml @@ -0,0 +1,29 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] + output-dir = "/tmp/tidb_cdc_test/csv_storage_update_pk_nonclustered/sync_diff-/output" + + source-instances = ["mysql1"] + + target-instance = "tidb0" + + target-check-tables = ["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/csv_storage_update_pk_nonclustered/data/prepare.sql b/tests/integration_tests/csv_storage_update_pk_nonclustered/data/prepare.sql new file mode 100644 index 00000000000..f3cd4ca4d24 --- /dev/null +++ b/tests/integration_tests/csv_storage_update_pk_nonclustered/data/prepare.sql @@ -0,0 +1,28 @@ +drop database if exists `test`; +create database `test`; +use `test`; + +CREATE TABLE `update_pk` ( + `id` int PRIMARY KEY NONCLUSTERED, + `pad` varchar(100) NOT NULL +); +INSERT INTO `update_pk` (`id`, `pad`) VALUES (1, 'example1'), (2, 'example2'); +INSERT INTO `update_pk` (`id`, `pad`) VALUES (10, 'example10'), (20, 'example20'); +INSERT INTO `update_pk` (`id`, `pad`) VALUES (100, 'example100'); +INSERT INTO `update_pk` (`id`, `pad`) VALUES (1000, 'example1000'); + + +SHOW INDEX FROM update_pk; + +CREATE TABLE `update_uk` ( + `id` int PRIMARY KEY NONCLUSTERED, + `uk` int NOT NULL, + `pad` varchar(100) NOT NULL, + UNIQUE KEY `uk` (`uk`) +); +INSERT INTO `update_uk` (`id`, `uk`, `pad`) VALUES (1, 1, 'example1'), (2, 2, 'example2'); +INSERT INTO `update_uk` (`id`, `uk`, `pad`) VALUES (10, 10, 'example10'), (20, 20, 'example20'); +INSERT INTO `update_uk` (`id`, `uk`, `pad`) VALUES (100, 100, 'example100'); +INSERT INTO `update_uk` (`id`, `uk`, `pad`) VALUES (1000, 1000, 'example1000'); + +SHOW INDEX FROM update_uk; \ No newline at end of file diff --git a/tests/integration_tests/csv_storage_update_pk_nonclustered/data/run.sql b/tests/integration_tests/csv_storage_update_pk_nonclustered/data/run.sql new file mode 100644 index 00000000000..86e7d7e7d77 --- /dev/null +++ b/tests/integration_tests/csv_storage_update_pk_nonclustered/data/run.sql @@ -0,0 +1,40 @@ +USE `test`; + +-- update_pk -- + +BEGIN; -- Note: multi-row exchange +UPDATE update_pk SET id = 3 WHERE id = 1; +UPDATE update_pk SET id = 1 WHERE id = 2; +UPDATE update_pk SET id = 2 WHERE id = 3; +COMMIT; + +BEGIN; -- Note: multi-row update with no order dependency +UPDATE update_pk SET id = 30 WHERE id = 10; +UPDATE update_pk SET id = 40 WHERE id = 20; +COMMIT; + +BEGIN; -- Single row update +UPDATE update_pk SET id = 200 WHERE id = 100; +COMMIT; + +-- Normal update +UPDATE update_pk SET pad='example1001' WHERE id = 1000; + +-- update_uk -- +BEGIN; -- Note: multi-row exchange +UPDATE update_uk SET uk = 3 WHERE uk = 1; +UPDATE update_uk SET uk = 1 WHERE uk = 2; +UPDATE update_uk SET uk = 2 WHERE uk = 3; +COMMIT; + +BEGIN; -- Note: multi-row update with no order dependency +UPDATE update_uk SET uk = 30 WHERE uk = 10; +UPDATE update_uk SET uk = 40 WHERE uk = 20; +COMMIT; + +BEGIN; -- Single row update +UPDATE update_uk SET uk = 200 WHERE uk = 100; +COMMIT; + +-- Normal update +UPDATE update_uk SET pad='example1001' WHERE uk = 1000; \ No newline at end of file diff --git a/tests/integration_tests/csv_storage_update_pk_nonclustered/result/changefeed1_pk.res b/tests/integration_tests/csv_storage_update_pk_nonclustered/result/changefeed1_pk.res new file mode 100644 index 00000000000..08f6eedb804 --- /dev/null +++ b/tests/integration_tests/csv_storage_update_pk_nonclustered/result/changefeed1_pk.res @@ -0,0 +1,24 @@ +"I","update_pk","test",450250823741472787,1,"example1" +"I","update_pk","test",450250823741472787,2,"example2" +"I","update_pk","test",450250823741472790,10,"example10" +"I","update_pk","test",450250823741472790,20,"example20" +"I","update_pk","test",450250823741472791,100,"example100" +"I","update_pk","test",450250823741472792,1000,"example1000" + +# split and sort in table sink +"D","update_pk","test",450250823807270922,1,"example1" +"D","update_pk","test",450250823807270922,2,"example2" +"I","update_pk","test",450250823807270922,2,"example1" +"I","update_pk","test",450250823807270922,1,"example2" + +# split and sort in table sink +"D","update_pk","test",450250823807270925,10,"example10" +"D","update_pk","test",450250823807270925,20,"example20" +"I","update_pk","test",450250823807270925,30,"example10" +"I","update_pk","test",450250823807270925,40,"example20" + +# split and sort in table sink +"D","update_pk","test",450250823807270927,100,"example100" +"I","update_pk","test",450250823807270927,200,"example100" + +"U","update_pk","test",450250823807270928,1000,"example1001" diff --git a/tests/integration_tests/csv_storage_update_pk_nonclustered/result/changefeed1_uk.res b/tests/integration_tests/csv_storage_update_pk_nonclustered/result/changefeed1_uk.res new file mode 100644 index 00000000000..b26f2219af2 --- /dev/null +++ b/tests/integration_tests/csv_storage_update_pk_nonclustered/result/changefeed1_uk.res @@ -0,0 +1,24 @@ +"I","update_uk","test",450250823780794385,1,1,"example1" +"I","update_uk","test",450250823780794385,2,2,"example2" +"I","update_uk","test",450250823780794387,10,10,"example10" +"I","update_uk","test",450250823780794387,20,20,"example20" +"I","update_uk","test",450250823780794389,100,100,"example100" +"I","update_uk","test",450250823780794390,1000,1000,"example1000" + +# split and sort in table sink +"D","update_uk","test",450250823807270931,1,1,"example1" +"D","update_uk","test",450250823807270931,2,2,"example2" +"I","update_uk","test",450250823807270931,1,2,"example1" +"I","update_uk","test",450250823807270931,2,1,"example2" + +# split and sort in table sink +"D","update_uk","test",450250823820115970,10,10,"example10" +"D","update_uk","test",450250823820115970,20,20,"example20" +"I","update_uk","test",450250823820115970,10,30,"example10" +"I","update_uk","test",450250823820115970,20,40,"example20" + +# split and sort in table sink +"D","update_uk","test",450250823820115973,100,100,"example100" +"I","update_uk","test",450250823820115973,100,200,"example100" + +"U","update_uk","test",450250823820115977,1000,1000,"example1001" diff --git a/tests/integration_tests/csv_storage_update_pk_nonclustered/result/changefeed2_pk.res b/tests/integration_tests/csv_storage_update_pk_nonclustered/result/changefeed2_pk.res new file mode 100644 index 00000000000..e2713a94f63 --- /dev/null +++ b/tests/integration_tests/csv_storage_update_pk_nonclustered/result/changefeed2_pk.res @@ -0,0 +1,28 @@ +"I","update_pk","test",450250823741472787,false,1,"example1" +"I","update_pk","test",450250823741472787,false,2,"example2" +"I","update_pk","test",450250823741472790,false,10,"example10" +"I","update_pk","test",450250823741472790,false,20,"example20" +"I","update_pk","test",450250823741472791,false,100,"example100" +"I","update_pk","test",450250823741472792,false,1000,"example1000" + +# split in csv encoder +# DIFF_RES: REPLACE INTO `test`.`update_pk`(`id`,`pad`) VALUES (2,'example1'); +# lost id=2 since delete are not sorted before insert within single txn +"D","update_pk","test",450250823807270922,true,1,"example1" +"I","update_pk","test",450250823807270922,true,2,"example1" +"D","update_pk","test",450250823807270922,true,2,"example2" +"I","update_pk","test",450250823807270922,true,1,"example2" + +# split in csv encoder +"D","update_pk","test",450250823807270925,true,10,"example10" +"I","update_pk","test",450250823807270925,true,30,"example10" +"D","update_pk","test",450250823807270925,true,20,"example20" +"I","update_pk","test",450250823807270925,true,40,"example20" + +# split in csv encoder +"D","update_pk","test",450250823807270927,true,100,"example100" +"I","update_pk","test",450250823807270927,true,200,"example100" + +# normal update event, also split in csv encoder +"D","update_pk","test",450250823807270928,true,1000,"example1000" +"I","update_pk","test",450250823807270928,true,1000,"example1001" \ No newline at end of file diff --git a/tests/integration_tests/csv_storage_update_pk_nonclustered/result/changefeed2_uk.res b/tests/integration_tests/csv_storage_update_pk_nonclustered/result/changefeed2_uk.res new file mode 100644 index 00000000000..1783ee5a0dd --- /dev/null +++ b/tests/integration_tests/csv_storage_update_pk_nonclustered/result/changefeed2_uk.res @@ -0,0 +1,26 @@ +"I","update_uk","test",450250823780794385,false,1,1,"example1" +"I","update_uk","test",450250823780794385,false,2,2,"example2" +"I","update_uk","test",450250823780794387,false,10,10,"example10" +"I","update_uk","test",450250823780794387,false,20,20,"example20" +"I","update_uk","test",450250823780794389,false,100,100,"example100" +"I","update_uk","test",450250823780794390,false,1000,1000,"example1000" + +# split in csv encoder, data is consistent since delete by pk +"D","update_uk","test",450250823807270931,true,1,1,"example1" +"I","update_uk","test",450250823807270931,true,1,2,"example1" +"D","update_uk","test",450250823807270931,true,2,2,"example2" +"I","update_uk","test",450250823807270931,true,2,1,"example2" + +# split in csv encoder +"D","update_uk","test",450250823820115970,true,10,10,"example10" +"I","update_uk","test",450250823820115970,true,10,30,"example10" +"D","update_uk","test",450250823820115970,true,20,20,"example20" +"I","update_uk","test",450250823820115970,true,20,40,"example20" + +# split in csv encoder +"D","update_uk","test",450250823820115973,true,100,100,"example100" +"I","update_uk","test",450250823820115973,true,100,200,"example100" + +# normal update event, also split in csv encoder +"D","update_uk","test",450250823820115977,true,1000,1000,"example1000" +"I","update_uk","test",450250823820115977,true,1000,1000,"example1001" \ No newline at end of file diff --git a/tests/integration_tests/csv_storage_update_pk_nonclustered/run.sh b/tests/integration_tests/csv_storage_update_pk_nonclustered/run.sh new file mode 100644 index 00000000000..4a6b1b73fe2 --- /dev/null +++ b/tests/integration_tests/csv_storage_update_pk_nonclustered/run.sh @@ -0,0 +1,61 @@ +#!/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_changefeed() { + local changefeed_id=$1 + local start_ts=$2 + local expected_split_count=$3 + local should_pass_check=$4 + SINK_URI="file://$WORK_DIR/storage_test/$changefeed_id?flush-interval=5s" + run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --config=$CUR/conf/$changefeed_id.toml -c "$changefeed_id" + + run_storage_consumer $WORK_DIR $SINK_URI $CUR/conf/$changefeed_id.toml $changefeed_id + sleep 8 + + cp $CUR/conf/diff_config.toml $WORK_DIR/diff_config.toml + sed -i "s//$changefeed_id/" $WORK_DIR/diff_config.toml + if [[ $should_pass_check == true ]]; then + check_sync_diff $WORK_DIR $WORK_DIR/diff_config.toml 100 + else + check_sync_diff $WORK_DIR $WORK_DIR/diff_config.toml 30 && exit 1 || echo "check_sync_diff failed as expected for $changefeed_id" + fi + + real_split_count=$(grep "split update event" $WORK_DIR/cdc.log | wc -l) + if [[ $real_split_count -ne $expected_split_count ]]; then + echo "expected split count $expected_split_count, real split count $real_split_count" + exit 1 + fi + run_sql "drop database if exists test" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} +} + +function run() { + if [ "$SINK_TYPE" != "storage" ]; then + return + fi + + rm -rf $WORK_DIR && mkdir -p $WORK_DIR + start_tidb_cluster --workdir $WORK_DIR + cd $WORK_DIR + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + + start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) + + run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql_file $CUR/data/run.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} + + run_changefeed "changefeed1" $start_ts 10 true + # changefeed2 fail since delete events are not sorted + run_changefeed "changefeed2" $start_ts 10 false +} + +trap stop_tidb_cluster EXIT +run $* +check_logs $WORK_DIR +echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" diff --git a/tests/integration_tests/run_group.sh b/tests/integration_tests/run_group.sh index 04f01885f3d..c79b0f15f5a 100755 --- a/tests/integration_tests/run_group.sh +++ b/tests/integration_tests/run_group.sh @@ -18,7 +18,7 @@ kafka_only="kafka_big_messages kafka_compression kafka_messages kafka_sink_error kafka_only_protocol="canal_json_adapter_compatibility canal_json_basic canal_json_content_compatible multi_topics avro_basic canal_json_handle_key_only open_protocol_handle_key_only canal_json_claim_check open_protocol_claim_check" kafka_only_v2="kafka_big_txn_v2 kafka_big_messages_v2 multi_tables_ddl_v2 multi_topics_v2" -storage_only="lossy_ddl storage_csv_update" +storage_only="lossy_ddl storage_csv_update csv_storage_update_pk_clustered csv_storage_update_pk_nonclustered" storage_only_csv="storage_cleanup csv_storage_basic csv_storage_multi_tables_ddl csv_storage_partition_table" storage_only_canal_json="canal_json_storage_basic canal_json_storage_partition_table" diff --git a/tests/integration_tests/syncpoint/data/prepare.sql b/tests/integration_tests/syncpoint/data/prepare.sql new file mode 100644 index 00000000000..bd603480b8c --- /dev/null +++ b/tests/integration_tests/syncpoint/data/prepare.sql @@ -0,0 +1,9 @@ +-- create user and grant all privileges +-- 'test123456' encoded by base64 -> "dGVzdDEyMzQ1Ng==" +CREATE USER 'syncpoint'@'%' IDENTIFIED BY 'test123456'; + +-- 授予所有数据库的所有权限 +GRANT ALL PRIVILEGES ON *.* TO 'syncpoint'@'%' WITH GRANT OPTION; + +-- 应用权限更改 +FLUSH PRIVILEGES; diff --git a/tests/integration_tests/syncpoint/run.sh b/tests/integration_tests/syncpoint/run.sh index 18c659917c9..fb43a06139f 100755 --- a/tests/integration_tests/syncpoint/run.sh +++ b/tests/integration_tests/syncpoint/run.sh @@ -160,9 +160,11 @@ function run() { start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + run_sql_file $CUR/data/prepare.sql ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + # this test contains `set global tidb_external_ts = ?` , which requires super privilege, so we # can't use the normal user - SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" + SINK_URI="mysql://syncpoint:dGVzdDEyMzQ1Ng==@127.0.0.1:3306/?max-txn-row=1" run_sql "SET GLOBAL tidb_enable_external_ts_read = on;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --config="$CUR/conf/changefeed.toml"