diff --git a/.editorconfig b/.editorconfig new file mode 100644 index 00000000000..43c6a002cce --- /dev/null +++ b/.editorconfig @@ -0,0 +1,10 @@ +[*] +end_of_line = lf +insert_final_newline = true +charset = utf-8 + +# tab_size = 4 spaces +[*.go] +indent_style = tab +indent_size = 4 +trim_trailing_whitespace = true diff --git a/Makefile b/Makefile index 73f943ead52..a56a7fce176 100644 --- a/Makefile +++ b/Makefile @@ -39,11 +39,12 @@ FAILPOINT := bin/failpoint-ctl FAILPOINT_ENABLE := $$(echo $(FAILPOINT_DIR) | xargs $(FAILPOINT) enable >/dev/null) FAILPOINT_DISABLE := $$(find $(FAILPOINT_DIR) | xargs $(FAILPOINT) disable >/dev/null) -LDFLAGS += -X "$(CDC_PKG)/pkg/util.BuildTS=$(shell date -u '+%Y-%m-%d %H:%M:%S')" -LDFLAGS += -X "$(CDC_PKG)/pkg/util.GitHash=$(shell git rev-parse HEAD)" -LDFLAGS += -X "$(CDC_PKG)/pkg/util.ReleaseVersion=$(shell git describe --tags --dirty="-dev")" -LDFLAGS += -X "$(CDC_PKG)/pkg/util.GitBranch=$(shell git rev-parse --abbrev-ref HEAD)" -LDFLAGS += -X "$(CDC_PKG)/pkg/util.GoVersion=$(shell go version)" +RELEASE_VERSION ?= $(shell git describe --tags --dirty="-dev") +LDFLAGS += -X "$(CDC_PKG)/pkg/version.ReleaseVersion=$(RELEASE_VERSION)" +LDFLAGS += -X "$(CDC_PKG)/pkg/version.BuildTS=$(shell date -u '+%Y-%m-%d %H:%M:%S')" +LDFLAGS += -X "$(CDC_PKG)/pkg/version.GitHash=$(shell git rev-parse HEAD)" +LDFLAGS += -X "$(CDC_PKG)/pkg/version.GitBranch=$(shell git rev-parse --abbrev-ref HEAD)" +LDFLAGS += -X "$(CDC_PKG)/pkg/version.GoVersion=$(shell go version)" default: build buildsucc @@ -102,10 +103,10 @@ integration_test_build: check_failpoint_ctl integration_test: integration_test_mysql integration_test_mysql: check_third_party_binary - tests/run.sh $(CASE) mysql + tests/run.sh mysql $(CASE) integration_test_kafka: check_third_party_binary - tests/run.sh $(CASE) kafka + tests/run.sh kafka $(CASE) fmt: @echo "gofmt (simplify)" diff --git a/cdc/changefeed.go b/cdc/changefeed.go index fb5c1995640..296c3a44fc0 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -743,7 +743,7 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { minResolvedTs = appliedTs } if appliedTs != math.MaxUint64 { - log.Info("some operation is still unapplied", + log.Debug("some operation is still unapplied", zap.String("captureID", captureID), zap.Uint64("appliedTs", appliedTs), zap.Stringer("status", status)) diff --git a/cdc/http_status.go b/cdc/http_status.go index 3693d110064..23cb2dce82a 100644 --- a/cdc/http_status.go +++ b/cdc/http_status.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/ticdc/cdc/kv" cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/security" - "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/ticdc/pkg/version" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promhttp" "go.etcd.io/etcd/clientv3" @@ -128,8 +128,8 @@ func (s *Server) handleStatus(w http.ResponseWriter, req *http.Request) { s.ownerLock.RLock() defer s.ownerLock.RUnlock() st := status{ - Version: util.ReleaseVersion, - GitHash: util.GitHash, + Version: version.ReleaseVersion, + GitHash: version.GitHash, Pid: os.Getpid(), } if s.capture != nil { diff --git a/cdc/kv/client.go b/cdc/kv/client.go index 8e2aad17c49..bb781790174 100644 --- a/cdc/kv/client.go +++ b/cdc/kv/client.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/ticdc/pkg/security" "github.com/pingcap/ticdc/pkg/txnutil" "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/ticdc/pkg/version" tidbkv "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/oracle" @@ -95,10 +96,15 @@ type regionErrorInfo struct { err error } +type regionEvent struct { + changeEvent *cdcpb.Event + resolvedTs *cdcpb.ResolvedTs +} + type regionFeedState struct { sri singleRegionInfo requestID uint64 - regionEventCh chan *cdcpb.Event + regionEventCh chan *regionEvent stopped int32 } @@ -106,7 +112,7 @@ func newRegionFeedState(sri singleRegionInfo, requestID uint64) *regionFeedState return ®ionFeedState{ sri: sri, requestID: requestID, - regionEventCh: make(chan *cdcpb.Event, 16), + regionEventCh: make(chan *regionEvent, 16), stopped: 0, } } @@ -343,7 +349,7 @@ func (c *CDCClient) newStream(ctx context.Context, addr string, storeID uint64) log.Info("get connection to store failed, retry later", zap.String("addr", addr), zap.Error(err)) return errors.Trace(err) } - err = util.CheckStoreVersion(ctx, c.pd, storeID) + err = version.CheckStoreVersion(ctx, c.pd, storeID) if err != nil { log.Error("check tikv version failed", zap.Error(err), zap.Uint64("storeID", storeID)) return errors.Trace(err) @@ -491,7 +497,10 @@ func (s *eventFeedSession) eventFeed(ctx context.Context, ts uint64) error { return ctx.Err() case errInfo := <-s.errCh: s.errChSizeGauge.Dec() - s.handleError(ctx, errInfo, true) + err := s.handleError(ctx, errInfo, true) + if err != nil { + return err + } } } }) @@ -668,7 +677,8 @@ MainLoop: regionID := rpcCtx.Meta.GetId() req := &cdcpb.ChangeDataRequest{ Header: &cdcpb.Header{ - ClusterId: s.client.clusterID, + ClusterId: s.client.clusterID, + TicdcVersion: version.ReleaseSemver(), }, RegionId: regionID, RequestId: requestID, @@ -946,7 +956,7 @@ func (s *eventFeedSession) divideAndSendEventFeedToRegions( // info will be sent to `regionCh`. // CAUTION: Note that this should only be invoked in a context that the region is not locked, otherwise use onRegionFail // instead. -func (s *eventFeedSession) handleError(ctx context.Context, errInfo regionErrorInfo, blocking bool) { +func (s *eventFeedSession) handleError(ctx context.Context, errInfo regionErrorInfo, blocking bool) error { err := errInfo.err switch eerr := errors.Cause(err).(type) { case *eventError: @@ -959,15 +969,21 @@ func (s *eventFeedSession) handleError(ctx context.Context, errInfo regionErrorI // TODO: If only confver is updated, we don't need to reload the region from region cache. metricFeedEpochNotMatchCounter.Inc() s.scheduleDivideRegionAndRequest(ctx, errInfo.span, errInfo.ts, blocking) - return + return nil } else if innerErr.GetRegionNotFound() != nil { metricFeedRegionNotFoundCounter.Inc() s.scheduleDivideRegionAndRequest(ctx, errInfo.span, errInfo.ts, blocking) - return + return nil } else if duplicatedRequest := innerErr.GetDuplicateRequest(); duplicatedRequest != nil { metricFeedDuplicateRequestCounter.Inc() log.Fatal("tikv reported duplicated request to the same region, which is not expected", zap.Uint64("regionID", duplicatedRequest.RegionId)) + return nil + } else if compatibility := innerErr.GetCompatibility(); compatibility != nil { + log.Error("tikv reported compatibility error, which is not expected", + zap.Uint64("storeID", errInfo.rpcCtx.GetStoreID()), + zap.Stringer("error", compatibility)) + return cerror.ErrVersionIncompatible.GenWithStackByArgs(compatibility) } else { metricFeedUnknownErrorCounter.Inc() log.Warn("receive empty or unknown error msg", zap.Stringer("error", innerErr)) @@ -975,7 +991,7 @@ func (s *eventFeedSession) handleError(ctx context.Context, errInfo regionErrorI case *rpcCtxUnavailableErr: metricFeedRPCCtxUnavailable.Inc() s.scheduleDivideRegionAndRequest(ctx, errInfo.span, errInfo.ts, blocking) - return + return nil default: bo := tikv.NewBackoffer(ctx, tikvRequestMaxBackoff) if errInfo.rpcCtx.Meta != nil { @@ -984,6 +1000,7 @@ func (s *eventFeedSession) handleError(ctx context.Context, errInfo regionErrorI } s.scheduleRegionRequest(ctx, errInfo.singleRegionInfo, blocking) + return nil } func (s *eventFeedSession) getRPCContextForRegion(ctx context.Context, id tikv.RegionVerID) (*tikv.RPCContext, error) { @@ -1023,6 +1040,10 @@ func (s *eventFeedSession) receiveFromStream( } }() + captureAddr := util.CaptureAddrFromCtx(ctx) + changefeedID := util.ChangefeedIDFromCtx(ctx) + metricSendEventBatchResolvedSize := batchResolvedEventSize.WithLabelValues(captureAddr, changefeedID) + // Each region has it's own goroutine to handle its messages. `regionStates` stores states of these regions. regionStates := make(map[uint64]*regionFeedState) @@ -1077,64 +1098,119 @@ func (s *eventFeedSession) receiveFromStream( } for _, event := range cevent.Events { - state, ok := regionStates[event.RegionId] - // Every region's range is locked before sending requests and unlocked after exiting, and the requestID - // is allocated while holding the range lock. Therefore the requestID is always incrementing. If a region - // is receiving messages with different requestID, only the messages with the larges requestID is valid. - isNewSubscription := !ok - if ok { - if state.requestID < event.RequestId { - log.Debug("region state entry will be replaced because received message of newer requestID", - zap.Uint64("regionID", event.RegionId), - zap.Uint64("oldRequestID", state.requestID), - zap.Uint64("requestID", event.RequestId), - zap.String("addr", addr)) - isNewSubscription = true - } else if state.requestID > event.RequestId { - log.Warn("drop event due to event belongs to a stale request", - zap.Uint64("regionID", event.RegionId), - zap.Uint64("requestID", event.RequestId), - zap.Uint64("currRequestID", state.requestID), - zap.String("addr", addr)) - continue - } + err = s.sendRegionChangeEvent(ctx, g, event, regionStates, pendingRegions, addr, limiter) + if err != nil { + return err + } + } + if cevent.ResolvedTs != nil { + metricSendEventBatchResolvedSize.Observe(float64(len(cevent.ResolvedTs.Regions))) + err = s.sendResolvedTs(ctx, g, cevent.ResolvedTs, regionStates, pendingRegions, addr) + if err != nil { + return err } + } + } +} - if isNewSubscription { - // It's the first response for this region. If the region is newly connected, the region info should - // have been put in `pendingRegions`. So here we load the region info from `pendingRegions` and start - // a new goroutine to handle messages from this region. - // Firstly load the region info. - state, ok = pendingRegions.take(event.RequestId) - if !ok { - log.Error("received an event but neither pending region nor running region was found", - zap.Uint64("regionID", event.RegionId), - zap.Uint64("requestID", event.RequestId), - zap.String("addr", addr)) - return cerror.ErrNoPendingRegion.GenWithStackByArgs(event.RegionId, event.RequestId, addr) - } +func (s *eventFeedSession) sendRegionChangeEvent( + ctx context.Context, + g *errgroup.Group, + event *cdcpb.Event, + regionStates map[uint64]*regionFeedState, + pendingRegions *syncRegionFeedStateMap, + addr string, + limiter *rate.Limiter, +) error { + state, ok := regionStates[event.RegionId] + // Every region's range is locked before sending requests and unlocked after exiting, and the requestID + // is allocated while holding the range lock. Therefore the requestID is always incrementing. If a region + // is receiving messages with different requestID, only the messages with the larges requestID is valid. + isNewSubscription := !ok + if ok { + if state.requestID < event.RequestId { + log.Debug("region state entry will be replaced because received message of newer requestID", + zap.Uint64("regionID", event.RegionId), + zap.Uint64("oldRequestID", state.requestID), + zap.Uint64("requestID", event.RequestId), + zap.String("addr", addr)) + isNewSubscription = true + } else if state.requestID > event.RequestId { + log.Warn("drop event due to event belongs to a stale request", + zap.Uint64("regionID", event.RegionId), + zap.Uint64("requestID", event.RequestId), + zap.Uint64("currRequestID", state.requestID), + zap.String("addr", addr)) + return nil + } + } - // Then spawn the goroutine to process messages of this region. - regionStates[event.RegionId] = state + if isNewSubscription { + // It's the first response for this region. If the region is newly connected, the region info should + // have been put in `pendingRegions`. So here we load the region info from `pendingRegions` and start + // a new goroutine to handle messages from this region. + // Firstly load the region info. + state, ok = pendingRegions.take(event.RequestId) + if !ok { + log.Error("received an event but neither pending region nor running region was found", + zap.Uint64("regionID", event.RegionId), + zap.Uint64("requestID", event.RequestId), + zap.String("addr", addr)) + return cerror.ErrNoPendingRegion.GenWithStackByArgs(event.RegionId, event.RequestId, addr) + } - g.Go(func() error { - return s.partialRegionFeed(ctx, state, limiter) - }) - } else if state.isStopped() { - log.Warn("drop event due to region feed stopped", - zap.Uint64("regionID", event.RegionId), - zap.Uint64("requestID", event.RequestId), + // Then spawn the goroutine to process messages of this region. + regionStates[event.RegionId] = state + + g.Go(func() error { + return s.partialRegionFeed(ctx, state, limiter) + }) + } else if state.isStopped() { + log.Warn("drop event due to region feed stopped", + zap.Uint64("regionID", event.RegionId), + zap.Uint64("requestID", event.RequestId), + zap.String("addr", addr)) + return nil + } + + select { + case state.regionEventCh <- ®ionEvent{ + changeEvent: event, + }: + case <-ctx.Done(): + return ctx.Err() + } + return nil +} + +func (s *eventFeedSession) sendResolvedTs( + ctx context.Context, + g *errgroup.Group, + resolvedTs *cdcpb.ResolvedTs, + regionStates map[uint64]*regionFeedState, + pendingRegions *syncRegionFeedStateMap, + addr string, +) error { + for _, regionID := range resolvedTs.Regions { + state, ok := regionStates[regionID] + if ok { + if state.isStopped() { + log.Warn("drop resolved ts due to region feed stopped", + zap.Uint64("regionID", regionID), + zap.Uint64("requestID", state.requestID), zap.String("addr", addr)) - continue + return nil } - select { - case state.regionEventCh <- event: + case state.regionEventCh <- ®ionEvent{ + resolvedTs: resolvedTs, + }: case <-ctx.Done(): return ctx.Err() } } } + return nil } // singleEventFeed handles events of a single EventFeed stream. @@ -1147,7 +1223,7 @@ func (s *eventFeedSession) singleEventFeed( regionID uint64, span regionspan.ComparableSpan, startTs uint64, - receiverCh <-chan *cdcpb.Event, + receiverCh <-chan *regionEvent, ) (uint64, error) { captureAddr := util.CaptureAddrFromCtx(ctx) changefeedID := util.ChangefeedIDFromCtx(ctx) @@ -1157,7 +1233,7 @@ func (s *eventFeedSession) singleEventFeed( metricPullEventCommitCounter := pullEventCounter.WithLabelValues(cdcpb.Event_COMMIT.String(), captureAddr, changefeedID) metricPullEventPrewriteCounter := pullEventCounter.WithLabelValues(cdcpb.Event_PREWRITE.String(), captureAddr, changefeedID) metricPullEventRollbackCounter := pullEventCounter.WithLabelValues(cdcpb.Event_ROLLBACK.String(), captureAddr, changefeedID) - metricSendEventResolvedCounter := sendEventCounter.WithLabelValues("native resolved", captureAddr, changefeedID) + metricSendEventResolvedCounter := sendEventCounter.WithLabelValues("native-resolved", captureAddr, changefeedID) metricSendEventCommitCounter := sendEventCounter.WithLabelValues("commit", captureAddr, changefeedID) metricSendEventCommittedCounter := sendEventCounter.WithLabelValues("committed", captureAddr, changefeedID) @@ -1169,6 +1245,37 @@ func (s *eventFeedSession) singleEventFeed( lastReceivedEventTime := time.Now() startFeedTime := time.Now() lastResolvedTs := startTs + handleResolvedTs := func(resolvedTs uint64) error { + if !initialized { + return nil + } + if resolvedTs < lastResolvedTs { + log.Warn("The resolvedTs is fallen back in kvclient", + zap.String("Event Type", "RESOLVED"), + zap.Uint64("resolvedTs", resolvedTs), + zap.Uint64("lastResolvedTs", lastResolvedTs), + zap.Uint64("regionID", regionID)) + return nil + } + // emit a checkpointTs + revent := &model.RegionFeedEvent{ + RegionID: regionID, + Resolved: &model.ResolvedSpan{ + Span: span, + ResolvedTs: resolvedTs, + }, + } + lastResolvedTs = resolvedTs + + select { + case s.eventCh <- revent: + metricSendEventResolvedCounter.Inc() + case <-ctx.Done(): + return errors.Trace(ctx.Err()) + } + return nil + } + select { case s.eventCh <- &model.RegionFeedEvent{ RegionID: regionID, @@ -1181,7 +1288,7 @@ func (s *eventFeedSession) singleEventFeed( return lastResolvedTs, errors.Trace(ctx.Err()) } for { - var event *cdcpb.Event + var event *regionEvent var ok bool select { case <-ctx.Done(): @@ -1231,151 +1338,134 @@ func (s *eventFeedSession) singleEventFeed( return lastResolvedTs, cerror.ErrEventFeedAborted.GenWithStackByArgs() } lastReceivedEventTime = time.Now() + if event.changeEvent != nil { + metricEventSize.Observe(float64(event.changeEvent.Event.Size())) + switch x := event.changeEvent.Event.(type) { + case *cdcpb.Event_Entries_: + for _, entry := range x.Entries.GetEntries() { + switch entry.Type { + case cdcpb.Event_INITIALIZED: + if time.Since(startFeedTime) > 20*time.Second { + log.Warn("The time cost of initializing is too mush", + zap.Duration("timeCost", time.Since(startFeedTime)), + zap.Uint64("regionID", regionID)) + } + metricPullEventInitializedCounter.Inc() + initialized = true + for _, cacheEntry := range matcher.cachedCommit { + value, ok := matcher.matchRow(cacheEntry) + if !ok { + // when cdc receives a commit log without a corresponding + // prewrite log before initialized, a committed log with + // the same key and start-ts must have been received. + log.Info("ignore commit event without prewrite", + zap.Binary("key", cacheEntry.GetKey()), + zap.Uint64("ts", cacheEntry.GetStartTs())) + continue + } + + revent, err := assembleCommitEvent(regionID, cacheEntry, value) + if err != nil { + return lastResolvedTs, errors.Trace(err) + } + select { + case s.eventCh <- revent: + metricSendEventCommitCounter.Inc() + case <-ctx.Done(): + return lastResolvedTs, errors.Trace(ctx.Err()) + } + } + matcher.clearCacheCommit() + case cdcpb.Event_COMMITTED: + metricPullEventCommittedCounter.Inc() + var opType model.OpType + switch entry.GetOpType() { + case cdcpb.Event_Row_DELETE: + opType = model.OpTypeDelete + case cdcpb.Event_Row_PUT: + opType = model.OpTypePut + default: + return lastResolvedTs, cerror.ErrUnknownKVEventType.GenWithStackByArgs(entry.GetOpType(), entry) + } - metricEventSize.Observe(float64(event.Event.Size())) - switch x := event.Event.(type) { - case *cdcpb.Event_Entries_: - for _, entry := range x.Entries.GetEntries() { - switch entry.Type { - case cdcpb.Event_INITIALIZED: - if time.Since(startFeedTime) > 20*time.Second { - log.Warn("The time cost of initializing is too mush", - zap.Duration("timeCost", time.Since(startFeedTime)), - zap.Uint64("regionID", regionID)) - } - metricPullEventInitializedCounter.Inc() - initialized = true - for _, cacheEntry := range matcher.cachedCommit { - value, ok := matcher.matchRow(cacheEntry) + revent := &model.RegionFeedEvent{ + RegionID: regionID, + Val: &model.RawKVEntry{ + OpType: opType, + Key: entry.Key, + Value: entry.GetValue(), + OldValue: entry.GetOldValue(), + StartTs: entry.StartTs, + CRTs: entry.CommitTs, + RegionID: regionID, + }, + } + + if entry.CommitTs <= lastResolvedTs { + log.Fatal("The CommitTs must be greater than the resolvedTs", + zap.String("Event Type", "COMMITTED"), + zap.Uint64("CommitTs", entry.CommitTs), + zap.Uint64("resolvedTs", lastResolvedTs), + zap.Uint64("regionID", regionID)) + } + select { + case s.eventCh <- revent: + metricSendEventCommittedCounter.Inc() + case <-ctx.Done(): + return lastResolvedTs, errors.Trace(ctx.Err()) + } + case cdcpb.Event_PREWRITE: + metricPullEventPrewriteCounter.Inc() + matcher.putPrewriteRow(entry) + case cdcpb.Event_COMMIT: + metricPullEventCommitCounter.Inc() + if entry.CommitTs <= lastResolvedTs { + log.Fatal("The CommitTs must be greater than the resolvedTs", + zap.String("Event Type", "COMMIT"), + zap.Uint64("CommitTs", entry.CommitTs), + zap.Uint64("resolvedTs", lastResolvedTs), + zap.Uint64("regionID", regionID)) + } + // emit a value + value, ok := matcher.matchRow(entry) if !ok { - // when cdc receives a commit log without a corresponding - // prewrite log before initialized, a committed log with - // the same key and start-ts must have been received. - log.Info("ignore commit event without prewrite", - zap.Binary("key", cacheEntry.GetKey()), - zap.Uint64("ts", cacheEntry.GetStartTs())) - continue + if !initialized { + matcher.cacheCommitRow(entry) + continue + } + return lastResolvedTs, cerror.ErrPrewriteNotMatch.GenWithStackByArgs(entry.GetKey(), entry.GetStartTs()) } - revent, err := assembleCommitEvent(regionID, cacheEntry, value) + revent, err := assembleCommitEvent(regionID, entry, value) if err != nil { return lastResolvedTs, errors.Trace(err) } + select { case s.eventCh <- revent: metricSendEventCommitCounter.Inc() case <-ctx.Done(): return lastResolvedTs, errors.Trace(ctx.Err()) } + case cdcpb.Event_ROLLBACK: + metricPullEventRollbackCounter.Inc() + matcher.rollbackRow(entry) } - matcher.clearCacheCommit() - case cdcpb.Event_COMMITTED: - metricPullEventCommittedCounter.Inc() - var opType model.OpType - switch entry.GetOpType() { - case cdcpb.Event_Row_DELETE: - opType = model.OpTypeDelete - case cdcpb.Event_Row_PUT: - opType = model.OpTypePut - default: - return lastResolvedTs, cerror.ErrUnknownKVEventType.GenWithStackByArgs(entry.GetOpType(), entry) - } - - revent := &model.RegionFeedEvent{ - RegionID: regionID, - Val: &model.RawKVEntry{ - OpType: opType, - Key: entry.Key, - Value: entry.GetValue(), - OldValue: entry.GetOldValue(), - StartTs: entry.StartTs, - CRTs: entry.CommitTs, - RegionID: regionID, - }, - } - - if entry.CommitTs <= lastResolvedTs { - log.Fatal("The CommitTs must be greater than the resolvedTs", - zap.String("Event Type", "COMMITTED"), - zap.Uint64("CommitTs", entry.CommitTs), - zap.Uint64("resolvedTs", lastResolvedTs), - zap.Uint64("regionID", regionID)) - } - select { - case s.eventCh <- revent: - metricSendEventCommittedCounter.Inc() - case <-ctx.Done(): - return lastResolvedTs, errors.Trace(ctx.Err()) - } - case cdcpb.Event_PREWRITE: - metricPullEventPrewriteCounter.Inc() - matcher.putPrewriteRow(entry) - case cdcpb.Event_COMMIT: - metricPullEventCommitCounter.Inc() - if entry.CommitTs <= lastResolvedTs { - log.Fatal("The CommitTs must be greater than the resolvedTs", - zap.String("Event Type", "COMMIT"), - zap.Uint64("CommitTs", entry.CommitTs), - zap.Uint64("resolvedTs", lastResolvedTs), - zap.Uint64("regionID", regionID)) - } - // emit a value - value, ok := matcher.matchRow(entry) - if !ok { - if !initialized { - matcher.cacheCommitRow(entry) - continue - } - return lastResolvedTs, cerror.ErrPrewriteNotMatch.GenWithStackByArgs(entry.GetKey(), entry.GetStartTs()) - } - - revent, err := assembleCommitEvent(regionID, entry, value) - if err != nil { - return lastResolvedTs, errors.Trace(err) - } - - select { - case s.eventCh <- revent: - metricSendEventCommitCounter.Inc() - case <-ctx.Done(): - return lastResolvedTs, errors.Trace(ctx.Err()) - } - case cdcpb.Event_ROLLBACK: - metricPullEventRollbackCounter.Inc() - matcher.rollbackRow(entry) + } + case *cdcpb.Event_Admin_: + log.Info("receive admin event", zap.Stringer("event", event.changeEvent)) + case *cdcpb.Event_Error: + return lastResolvedTs, cerror.WrapError(cerror.ErrEventFeedEventError, &eventError{err: x.Error}) + case *cdcpb.Event_ResolvedTs: + if err := handleResolvedTs(x.ResolvedTs); err != nil { + return lastResolvedTs, errors.Trace(err) } } - case *cdcpb.Event_Admin_: - log.Info("receive admin event", zap.Stringer("event", event)) - case *cdcpb.Event_Error: - return lastResolvedTs, cerror.WrapError(cerror.ErrEventFeedEventError, &eventError{err: x.Error}) - case *cdcpb.Event_ResolvedTs: - if !initialized { - continue - } - if x.ResolvedTs < lastResolvedTs { - log.Warn("The resolvedTs is fallen back in kvclient", - zap.String("Event Type", "RESOLVED"), - zap.Uint64("resolvedTs", x.ResolvedTs), - zap.Uint64("lastResolvedTs", lastResolvedTs), - zap.Uint64("regionID", regionID)) - continue - } - // emit a checkpointTs - revent := &model.RegionFeedEvent{ - RegionID: regionID, - Resolved: &model.ResolvedSpan{ - Span: span, - ResolvedTs: x.ResolvedTs, - }, - } - lastResolvedTs = x.ResolvedTs + } - select { - case s.eventCh <- revent: - metricSendEventResolvedCounter.Inc() - case <-ctx.Done(): - return lastResolvedTs, errors.Trace(ctx.Err()) + if event.resolvedTs != nil { + if err := handleResolvedTs(event.resolvedTs.Ts); err != nil { + return lastResolvedTs, errors.Trace(err) } } } diff --git a/cdc/kv/client_test.go b/cdc/kv/client_test.go index da78b8303ea..f9aec45bca4 100644 --- a/cdc/kv/client_test.go +++ b/cdc/kv/client_test.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/ticdc/pkg/regionspan" "github.com/pingcap/ticdc/pkg/security" "github.com/pingcap/ticdc/pkg/txnutil" - "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/ticdc/pkg/version" "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv" pd "github.com/tikv/pd/client" @@ -140,7 +140,7 @@ func (s *etcdSuite) TestConnectOfflineTiKV(c *check.C) { mvccStore := mocktikv.MustNewMVCCStore() rpcClient, pdClient, err := mocktikv.NewTiKVAndPDClient(cluster, mvccStore, "") c.Assert(err, check.IsNil) - pdClient = &mockPDClient{Client: pdClient, version: util.MinTiKVVersion.String()} + pdClient = &mockPDClient{Client: pdClient, version: version.MinTiKVVersion.String()} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) c.Assert(err, check.IsNil) @@ -217,7 +217,7 @@ func (s *etcdSuite) TestRecvLargeMessageSize(c *check.C) { mvccStore := mocktikv.MustNewMVCCStore() rpcClient, pdClient, err := mocktikv.NewTiKVAndPDClient(cluster, mvccStore, "") c.Assert(err, check.IsNil) - pdClient = &mockPDClient{Client: pdClient, version: util.MinTiKVVersion.String()} + pdClient = &mockPDClient{Client: pdClient, version: version.MinTiKVVersion.String()} kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) c.Assert(err, check.IsNil) diff --git a/cdc/kv/metrics.go b/cdc/kv/metrics.go index 5c0854766b7..07789f5bfaa 100644 --- a/cdc/kv/metrics.go +++ b/cdc/kv/metrics.go @@ -67,6 +67,14 @@ var ( Name: "channel_size", Help: "size of each channel in kv client", }, []string{"id", "channel"}) + batchResolvedEventSize = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: "ticdc", + Subsystem: "kvclient", + Name: "batch_resolved_event_size", + Help: "The number of region in one batch resolved ts event", + Buckets: prometheus.ExponentialBuckets(2, 2, 16), + }, []string{"capture", "changefeed"}) etcdRequestCounter = prometheus.NewCounterVec( prometheus.CounterOpts{ Namespace: "ticdc", @@ -85,5 +93,6 @@ func InitMetrics(registry *prometheus.Registry) { registry.MustRegister(pullEventCounter) registry.MustRegister(sendEventCounter) registry.MustRegister(clientChannelSize) + registry.MustRegister(batchResolvedEventSize) registry.MustRegister(etcdRequestCounter) } diff --git a/cdc/model/owner.go b/cdc/model/owner.go index e281bc1e4ce..82d535bfe2f 100644 --- a/cdc/model/owner.go +++ b/cdc/model/owner.go @@ -123,13 +123,34 @@ type MoveTableJob struct { Status MoveTableStatus } +// All TableOperation status +const ( + OperDispatched uint64 = iota + OperProcessed + OperFinished +) + // TableOperation records the current information of a table migration type TableOperation struct { Delete bool `json:"delete"` // if the operation is a delete operation, BoundaryTs is checkpoint ts // if the operation is a add operation, BoundaryTs is start ts BoundaryTs uint64 `json:"boundary_ts"` - Done bool `json:"done"` + Done bool `json:"done"` // deprecated, will be removed in the next version + Status uint64 `json:"status,omitempty"` +} + +// TableProcessed returns whether the table has been processed by processor +func (o *TableOperation) TableProcessed() bool { + // TODO: remove o.Done + return o.Status == OperProcessed || o.Status == OperFinished || o.Done +} + +// TableApplied returns whether the table has finished the startup procedure. +// Returns true if table has been processed by processor and resolved ts reaches global resolved ts. +func (o *TableOperation) TableApplied() bool { + // TODO: remove o.Done + return o.Status == OperFinished || o.Done } // Clone returns a deep-clone of the struct @@ -232,13 +253,14 @@ func (ts *TaskStatus) AddTable(id TableID, table *TableReplicaInfo, boundaryTs T ts.Operation[id] = &TableOperation{ Delete: false, BoundaryTs: boundaryTs, + Status: OperDispatched, } } // SomeOperationsUnapplied returns true if there are some operations not applied func (ts *TaskStatus) SomeOperationsUnapplied() bool { for _, o := range ts.Operation { - if !o.Done { + if !o.TableApplied() { return true } } @@ -249,7 +271,7 @@ func (ts *TaskStatus) SomeOperationsUnapplied() bool { func (ts *TaskStatus) AppliedTs() Ts { appliedTs := uint64(math.MaxUint64) for _, o := range ts.Operation { - if !o.Done { + if !o.TableApplied() { if appliedTs > o.BoundaryTs { appliedTs = o.BoundaryTs } diff --git a/cdc/model/sink.go b/cdc/model/sink.go index d4e091169d2..8f4d764dc10 100644 --- a/cdc/model/sink.go +++ b/cdc/model/sink.go @@ -236,6 +236,31 @@ func (r *RowChangedEvent) IsDelete() bool { return len(r.PreColumns) != 0 && len(r.Columns) == 0 } +// HandleKeyColumns returns the column(s) corresponding to the handle key(s) +func (r *RowChangedEvent) HandleKeyColumns() []*Column { + pkeyCols := make([]*Column, 0) + + var cols []*Column + if r.IsDelete() { + cols = r.PreColumns + } else { + cols = r.Columns + } + + for _, col := range cols { + if col != nil && col.Flag.IsHandleKey() { + pkeyCols = append(pkeyCols, col) + } + } + + if len(pkeyCols) == 0 { + // TODO redact the message + log.Fatal("Cannot find handle key columns, bug?", zap.Reflect("event", r)) + } + + return pkeyCols +} + // Column represents a column value in row changed event type Column struct { Name string `json:"name"` diff --git a/cdc/processor.go b/cdc/processor.go index 2a99265fb54..af98e21b9f4 100644 --- a/cdc/processor.go +++ b/cdc/processor.go @@ -584,7 +584,7 @@ func (p *processor) removeTable(tableID int64) { // handleTables handles table scheduler on this processor, add or remove table puller func (p *processor) handleTables(ctx context.Context, status *model.TaskStatus) (tablesToRemove []model.TableID, err error) { for tableID, opt := range status.Operation { - if opt.Done { + if opt.TableProcessed() { continue } if opt.Delete { @@ -593,6 +593,7 @@ func (p *processor) handleTables(ctx context.Context, status *model.TaskStatus) if !exist { log.Warn("table which will be deleted is not found", zap.Int64("tableID", tableID)) opt.Done = true + opt.Status = model.OperFinished continue } stopped, checkpointTs := table.safeStop() @@ -603,6 +604,7 @@ func (p *processor) handleTables(ctx context.Context, status *model.TaskStatus) if checkpointTs <= p.position.CheckPointTs { tablesToRemove = append(tablesToRemove, tableID) opt.Done = true + opt.Status = model.OperFinished } } } @@ -615,6 +617,7 @@ func (p *processor) handleTables(ctx context.Context, status *model.TaskStatus) return tablesToRemove, cerror.ErrProcessorTableNotFound.GenWithStack("normal table(%d) and mark table not match ", tableID) } p.addTable(ctx, tableID, replicaInfo) + opt.Status = model.OperProcessed } } @@ -631,6 +634,7 @@ func (p *processor) handleTables(ctx context.Context, status *model.TaskStatus) continue } status.Operation[tableID].Done = true + status.Operation[tableID].Status = model.OperFinished default: goto done } diff --git a/cdc/puller/puller.go b/cdc/puller/puller.go index e16b4f1efb1..7a88b026cd3 100644 --- a/cdc/puller/puller.go +++ b/cdc/puller/puller.go @@ -139,8 +139,17 @@ func (p *pullerImpl) Run(ctx context.Context) error { metricEventCounterKv := kvEventCounter.WithLabelValues(captureAddr, changefeedID, "kv") metricEventCounterResolved := kvEventCounter.WithLabelValues(captureAddr, changefeedID, "resolved") metricTxnCollectCounterKv := txnCollectCounter.WithLabelValues(captureAddr, changefeedID, tableName, "kv") - metricTxnCollectCounterResolved := txnCollectCounter.WithLabelValues(captureAddr, changefeedID, tableName, "kv") - + metricTxnCollectCounterResolved := txnCollectCounter.WithLabelValues(captureAddr, changefeedID, tableName, "resolved") + defer func() { + outputChanSizeGauge.DeleteLabelValues(captureAddr, changefeedID, tableName) + eventChanSizeGauge.DeleteLabelValues(captureAddr, changefeedID, tableName) + memBufferSizeGauge.DeleteLabelValues(captureAddr, changefeedID, tableName) + pullerResolvedTsGauge.DeleteLabelValues(captureAddr, changefeedID, tableName) + kvEventCounter.DeleteLabelValues(captureAddr, changefeedID, "kv") + kvEventCounter.DeleteLabelValues(captureAddr, changefeedID, "resolved") + txnCollectCounter.DeleteLabelValues(captureAddr, changefeedID, tableName, "kv") + txnCollectCounter.DeleteLabelValues(captureAddr, changefeedID, tableName, "resolved") + }() g.Go(func() error { for { select { diff --git a/cdc/server.go b/cdc/server.go index a6c079f9a40..1b42cbcb434 100644 --- a/cdc/server.go +++ b/cdc/server.go @@ -27,6 +27,7 @@ import ( cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/security" "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/ticdc/pkg/version" pd "github.com/tikv/pd/client" "go.etcd.io/etcd/mvcc" "go.uber.org/zap" @@ -227,7 +228,7 @@ func (s *Server) Run(ctx context.Context) error { // To not block CDC server startup, we need to warn instead of error // when TiKV is incompatible. errorTiKVIncompatible := false - err = util.CheckClusterVersion(ctx, s.pdClient, s.pdEndpoints[0], s.opts.credential, errorTiKVIncompatible) + err = version.CheckClusterVersion(ctx, s.pdClient, s.pdEndpoints[0], s.opts.credential, errorTiKVIncompatible) if err != nil { return err } diff --git a/cdc/sink/cdclog/file.go b/cdc/sink/cdclog/file.go index ca368f5a741..1e09eda6d9c 100644 --- a/cdc/sink/cdclog/file.go +++ b/cdc/sink/cdclog/file.go @@ -18,17 +18,16 @@ import ( "net/url" "os" "path/filepath" - "sync" - "time" + "github.com/pingcap/errors" "github.com/pingcap/log" parsemodel "github.com/pingcap/parser/model" + "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/cdc/sink/codec" cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/uber-go/atomic" "go.uber.org/zap" - "golang.org/x/sync/errgroup" ) const ( @@ -57,7 +56,7 @@ type tableStream struct { sendSize *atomic.Int64 } -func newTableStream(tableID int64) *tableStream { +func newTableStream(tableID int64) logUnit { return &tableStream{ tableID: tableID, dataCh: make(chan *model.RowChangedEvent, defaultBufferChanSize), @@ -67,19 +66,132 @@ func newTableStream(tableID int64) *tableStream { } } +func (ts *tableStream) dataChan() chan *model.RowChangedEvent { + return ts.dataCh +} + +func (ts *tableStream) TableID() int64 { + return ts.tableID +} + +func (ts *tableStream) Events() *atomic.Int64 { + return ts.sendEvents +} + +func (ts *tableStream) Size() *atomic.Int64 { + return ts.sendSize +} + +func (ts *tableStream) isEmpty() bool { + return ts.sendEvents.Load() == 0 +} + +func (ts *tableStream) shouldFlush() bool { + return ts.sendSize.Load() > maxPartFlushSize +} + +func (ts *tableStream) flush(ctx context.Context, sink *logSink) error { + var fileName string + flushedEvents := ts.sendEvents.Load() + flushedSize := ts.sendSize.Load() + if flushedEvents == 0 { + log.Info("[flushTableStreams] no events to flush") + return nil + } + firstCreated := false + if ts.encoder == nil { + // create encoder for each file + ts.encoder = sink.encoder() + firstCreated = true + } + for event := int64(0); event < flushedEvents; event++ { + row := <-ts.dataCh + if event == flushedEvents-1 { + // the last event + fileName = makeTableFileName(row.CommitTs) + } + _, err := ts.encoder.AppendRowChangedEvent(row) + if err != nil { + return err + } + } + rowDatas := ts.encoder.MixedBuild(firstCreated) + defer func() { + if ts.encoder != nil { + ts.encoder.Reset() + } + }() + + log.Debug("[flushTableStreams] build cdc log data", + zap.Int64("table id", ts.tableID), + zap.Int64("flushed size", flushedSize), + zap.Int64("flushed event", flushedEvents), + zap.Int("encode size", len(rowDatas)), + zap.String("file name", fileName), + ) + + tableDir := filepath.Join(sink.root(), makeTableDirectoryName(ts.tableID)) + + if ts.rowFile == nil { + // create new file to append data + err := os.MkdirAll(tableDir, defaultDirMode) + if err != nil { + return err + } + file, err := os.OpenFile(filepath.Join(tableDir, defaultFileName), os.O_CREATE|os.O_WRONLY|os.O_APPEND, defaultFileMode) + if err != nil { + return err + } + ts.rowFile = file + } + + _, err := ts.rowFile.Write(rowDatas) + if err != nil { + return err + } + + stat, err := ts.rowFile.Stat() + if err != nil { + return err + } + + if stat.Size() > maxRowFileSize { + // rotate file + err := ts.rowFile.Close() + if err != nil { + return err + } + oldPath := filepath.Join(tableDir, defaultFileName) + newPath := filepath.Join(tableDir, fileName) + err = os.Rename(oldPath, newPath) + if err != nil { + return err + } + file, err := os.OpenFile(filepath.Join(tableDir, defaultFileName), os.O_CREATE|os.O_WRONLY|os.O_APPEND, defaultFileMode) + if err != nil { + return err + } + ts.rowFile = file + ts.encoder = nil + } + + ts.sendEvents.Sub(flushedEvents) + ts.sendSize.Sub(flushedSize) + return nil +} + type fileSink struct { + *logSink + logMeta *logMeta logPath *logPath ddlFile *os.File - encoder func() codec.EventBatchEncoder - ddlEncoder codec.EventBatchEncoder - hashMap sync.Map - tableStreams []*tableStream + ddlEncoder codec.EventBatchEncoder } -func (f *fileSink) flushLogMeta(ctx context.Context) error { +func (f *fileSink) flushLogMeta() error { data, err := f.logMeta.Marshal() if err != nil { return cerror.WrapError(cerror.ErrMarshalFailed, err) @@ -93,99 +205,6 @@ func (f *fileSink) flushLogMeta(ctx context.Context) error { return cerror.WrapError(cerror.ErrFileSinkFileOp, err) } -func (f *fileSink) flushTableStreams(ctx context.Context) error { - // TODO use a fixed worker pool - eg, _ := errgroup.WithContext(ctx) - for _, ts := range f.tableStreams { - tsReplica := ts - eg.Go(func() error { - var fileName string - flushedEvents := tsReplica.sendEvents.Load() - flushedSize := tsReplica.sendSize.Load() - firstCreated := false - if tsReplica.encoder == nil { - // create encoder for each file - tsReplica.encoder = f.encoder() - firstCreated = true - } - for event := int64(0); event < flushedEvents; event++ { - row := <-tsReplica.dataCh - if event == flushedEvents-1 { - // the last event - fileName = makeTableFileName(row.CommitTs) - } - _, err := tsReplica.encoder.AppendRowChangedEvent(row) - if err != nil { - return err - } - } - rowDatas := tsReplica.encoder.MixedBuild(firstCreated) - defer func() { - if tsReplica.encoder != nil { - tsReplica.encoder.Reset() - } - }() - - log.Debug("[flushTableStreams] build cdc log data", - zap.Int64("table id", tsReplica.tableID), - zap.Int64("flushed size", flushedSize), - zap.Int64("flushed event", flushedEvents), - zap.Int("encode size", len(rowDatas)), - zap.String("file name", fileName), - ) - - tableDir := filepath.Join(f.logPath.root, makeTableDirectoryName(tsReplica.tableID)) - - if tsReplica.rowFile == nil { - // create new file to append data - err := os.MkdirAll(tableDir, defaultDirMode) - if err != nil { - return cerror.WrapError(cerror.ErrFileSinkCreateDir, err) - } - file, err := os.OpenFile(filepath.Join(tableDir, defaultFileName), os.O_CREATE|os.O_WRONLY|os.O_APPEND, defaultFileMode) - if err != nil { - return cerror.WrapError(cerror.ErrFileSinkFileOp, err) - } - tsReplica.rowFile = file - } - - stat, err := tsReplica.rowFile.Stat() - if err != nil { - return cerror.WrapError(cerror.ErrFileSinkFileOp, err) - } - - if stat.Size() > maxRowFileSize { - // rotate file - err := tsReplica.rowFile.Close() - if err != nil { - return cerror.WrapError(cerror.ErrFileSinkFileOp, err) - } - oldPath := filepath.Join(tableDir, defaultFileName) - newPath := filepath.Join(tableDir, fileName) - err = os.Rename(oldPath, newPath) - if err != nil { - return cerror.WrapError(cerror.ErrFileSinkFileOp, err) - } - file, err := os.OpenFile(filepath.Join(tableDir, defaultFileName), os.O_CREATE|os.O_WRONLY|os.O_APPEND, defaultFileMode) - if err != nil { - return cerror.WrapError(cerror.ErrFileSinkFileOp, err) - } - tsReplica.rowFile = file - tsReplica.encoder = nil - } - _, err = tsReplica.rowFile.Write(rowDatas) - if err != nil { - return cerror.WrapError(cerror.ErrFileSinkFileOp, err) - } - - tsReplica.sendEvents.Sub(flushedEvents) - tsReplica.sendSize.Sub(flushedSize) - return nil - }) - } - return eg.Wait() -} - func (f *fileSink) createDDLFile(commitTs uint64) (*os.File, error) { fileName := makeDDLFileName(commitTs) file, err := os.OpenFile(filepath.Join(f.logPath.ddl, fileName), os.O_CREATE|os.O_WRONLY|os.O_APPEND, defaultFileMode) @@ -197,62 +216,32 @@ func (f *fileSink) createDDLFile(commitTs uint64) (*os.File, error) { } func (f *fileSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { - for _, row := range rows { - // dispatch row event by tableID - tableID := row.Table.GetTableID() - var ( - ok bool - item interface{} - hash int - ) - if item, ok = f.hashMap.Load(tableID); !ok { - // found new tableID - f.tableStreams = append(f.tableStreams, newTableStream(tableID)) - hash = len(f.tableStreams) - 1 - f.hashMap.Store(tableID, hash) - } else { - hash = item.(int) - } - select { - case <-ctx.Done(): - return ctx.Err() - case f.tableStreams[hash].dataCh <- row: - f.tableStreams[hash].sendEvents.Inc() - f.tableStreams[hash].sendSize.Add(row.ApproximateSize) - } - } - return nil + return f.emitRowChangedEvents(ctx, newTableStream, rows...) } func (f *fileSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { log.Debug("[FlushRowChangedEvents] enter", zap.Uint64("ts", resolvedTs)) - // TODO update flush policy with size - select { - case <-ctx.Done(): - return 0, ctx.Err() - case <-time.After(defaultFlushRowChangedEventDuration): - return resolvedTs, f.flushTableStreams(ctx) - } + return f.flushRowChangedEvents(ctx, resolvedTs) } func (f *fileSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { log.Debug("[EmitCheckpointTs]", zap.Uint64("ts", ts)) f.logMeta.GlobalResolvedTS = ts - return f.flushLogMeta(ctx) + return f.flushLogMeta() } func (f *fileSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { switch ddl.Type { case parsemodel.ActionCreateTable: f.logMeta.Names[ddl.TableInfo.TableID] = model.QuoteSchema(ddl.TableInfo.Schema, ddl.TableInfo.Table) - err := f.flushLogMeta(ctx) + err := f.flushLogMeta() if err != nil { return err } case parsemodel.ActionRenameTable: delete(f.logMeta.Names, ddl.PreTableInfo.TableID) f.logMeta.Names[ddl.TableInfo.TableID] = model.QuoteSchema(ddl.TableInfo.Schema, ddl.TableInfo.Table) - err := f.flushLogMeta(ctx) + err := f.flushLogMeta() if err != nil { return err } @@ -355,7 +344,7 @@ func (f *fileSink) Close() error { } // NewLocalFileSink support log data to file. -func NewLocalFileSink(sinkURI *url.URL) (*fileSink, error) { +func NewLocalFileSink(ctx context.Context, sinkURI *url.URL, errCh chan error) (*fileSink, error) { log.Info("[NewLocalFileSink]", zap.String("host", sinkURI.Host), zap.String("path", sinkURI.Path), @@ -373,15 +362,22 @@ func NewLocalFileSink(sinkURI *url.URL) (*fileSink, error) { zap.Error(err)) return nil, cerror.WrapError(cerror.ErrFileSinkCreateDir, err) } - return &fileSink{ + + f := &fileSink{ logMeta: newLogMeta(), logPath: logPath, - encoder: func() codec.EventBatchEncoder { - ret := codec.NewJSONEventBatchEncoder() - ret.(*codec.JSONEventBatchEncoder).SetMixedBuildSupport(true) - return ret - }, - - tableStreams: make([]*tableStream, 0), - }, nil + logSink: newLogSink(logPath.root, nil), + } + + // important! we should flush asynchronously in another goroutine + go func() { + if err := f.startFlush(ctx); err != nil && errors.Cause(err) != context.Canceled { + select { + case <-ctx.Done(): + return + case errCh <- err: + } + } + }() + return f, nil } diff --git a/cdc/sink/cdclog/s3.go b/cdc/sink/cdclog/s3.go index 06e9514fe7f..47ea9e55ff5 100644 --- a/cdc/sink/cdclog/s3.go +++ b/cdc/sink/cdclog/s3.go @@ -17,7 +17,6 @@ import ( "context" "net/url" "strings" - "sync" "time" "github.com/pingcap/br/pkg/storage" @@ -30,11 +29,9 @@ import ( cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/uber-go/atomic" "go.uber.org/zap" - "golang.org/x/sync/errgroup" ) const ( - maxNotifySize = 15 << 20 // trigger flush if one table has reached 16Mb data size in memory maxPartFlushSize = 5 << 20 // The minimal multipart upload size is 5Mb. maxCompletePartSize = 100 << 20 // rotate row changed event file if one complete file larger than 100Mb maxDDLFlushSize = 10 << 20 // rotate ddl event file if one complete file larger than 10Mb @@ -59,11 +56,31 @@ type tableBuffer struct { } } -func (tb *tableBuffer) IsEmpty() bool { +func (tb *tableBuffer) dataChan() chan *model.RowChangedEvent { + return tb.dataCh +} + +func (tb *tableBuffer) TableID() int64 { + return tb.tableID +} + +func (tb *tableBuffer) Events() *atomic.Int64 { + return tb.sendEvents +} + +func (tb *tableBuffer) Size() *atomic.Int64 { + return tb.sendSize +} + +func (tb *tableBuffer) isEmpty() bool { return tb.sendEvents.Load() == 0 && tb.uploadParts.uploadNum == 0 } -func (tb *tableBuffer) flush(ctx context.Context, s *s3Sink) error { +func (tb *tableBuffer) shouldFlush() bool { + return tb.sendSize.Load() > maxPartFlushSize +} + +func (tb *tableBuffer) flush(ctx context.Context, sink *logSink) error { hashPart := tb.uploadParts sendEvents := tb.sendEvents.Load() if sendEvents == 0 && hashPart.uploadNum == 0 { @@ -74,7 +91,7 @@ func (tb *tableBuffer) flush(ctx context.Context, s *s3Sink) error { firstCreated := false if tb.encoder == nil { // create encoder for each file - tb.encoder = s.encoder() + tb.encoder = sink.encoder() firstCreated = true } @@ -109,13 +126,13 @@ func (tb *tableBuffer) flush(ctx context.Context, s *s3Sink) error { // zap.ByteString("rowDatas", rowDatas), ) - if len(rowDatas) > 0 { - if len(rowDatas) > maxPartFlushSize || hashPart.uploadNum > 0 { - // S3 multi-upload need every chunk(except the last one) is greater than 5Mb - // so, if this batch data size is greater than 5Mb or it has uploadPart already - // we will use multi-upload this batch data + if len(rowDatas) > maxPartFlushSize || hashPart.uploadNum > 0 { + // S3 multi-upload need every chunk(except the last one) is greater than 5Mb + // so, if this batch data size is greater than 5Mb or it has uploadPart already + // we will use multi-upload this batch data + if len(rowDatas) > 0 { if hashPart.uploader == nil { - uploader, err := s.storage.CreateUploader(ctx, newFileName) + uploader, err := sink.storage().CreateUploader(ctx, newFileName) if err != nil { return cerror.WrapError(cerror.ErrS3SinkStorageAPI, err) } @@ -129,29 +146,29 @@ func (tb *tableBuffer) flush(ctx context.Context, s *s3Sink) error { hashPart.byteSize += int64(len(rowDatas)) hashPart.uploadNum++ + } - if hashPart.byteSize > maxCompletePartSize || len(rowDatas) <= maxPartFlushSize { - // we need do complete when total upload size is greater than 100Mb - // or this part data is less than 5Mb to avoid meet EntityTooSmall error - log.Info("[FlushRowChangedEvents] complete file", zap.Int64("tableID", tb.tableID)) - err = hashPart.uploader.CompleteUpload(ctx) - if err != nil { - return cerror.WrapError(cerror.ErrS3SinkStorageAPI, err) - } - hashPart.byteSize = 0 - hashPart.uploadNum = 0 - hashPart.uploader = nil - tb.encoder = nil - } - } else { - // generate normal file because S3 multi-upload need every part at least 5Mb. - log.Info("[FlushRowChangedEvents] normal upload file", zap.Int64("tableID", tb.tableID)) - err := s.storage.Write(ctx, newFileName, rowDatas) + if hashPart.byteSize > maxCompletePartSize || len(rowDatas) <= maxPartFlushSize { + // we need do complete when total upload size is greater than 100Mb + // or this part data is less than 5Mb to avoid meet EntityTooSmall error + log.Info("[FlushRowChangedEvents] complete file", zap.Int64("tableID", tb.tableID)) + err := hashPart.uploader.CompleteUpload(ctx) if err != nil { return cerror.WrapError(cerror.ErrS3SinkStorageAPI, err) } + hashPart.byteSize = 0 + hashPart.uploadNum = 0 + hashPart.uploader = nil tb.encoder = nil } + } else { + // generate normal file because S3 multi-upload need every part at least 5Mb. + log.Info("[FlushRowChangedEvents] normal upload file", zap.Int64("tableID", tb.tableID)) + err := sink.storage().Write(ctx, newFileName, rowDatas) + if err != nil { + return cerror.WrapError(cerror.ErrS3SinkStorageAPI, err) + } + tb.encoder = nil } tb.sendEvents.Sub(sendEvents) @@ -160,7 +177,7 @@ func (tb *tableBuffer) flush(ctx context.Context, s *s3Sink) error { return nil } -func newTableBuffer(tableID int64) *tableBuffer { +func newTableBuffer(tableID int64) logUnit { return &tableBuffer{ tableID: tableID, dataCh: make(chan *model.RowChangedEvent, defaultBufferChanSize), @@ -179,58 +196,20 @@ func newTableBuffer(tableID int64) *tableBuffer { } type s3Sink struct { + *logSink + prefix string storage *storage.S3Storage logMeta *logMeta - encoder func() codec.EventBatchEncoder // hold encoder for ddl event log - ddlEncoder codec.EventBatchEncoder - hashMap sync.Map - tableBuffers []*tableBuffer - notifyChan chan struct{} + ddlEncoder codec.EventBatchEncoder } func (s *s3Sink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { - shouldFlush := false - for _, row := range rows { - // dispatch row event by tableID - tableID := row.Table.GetTableID() - var ( - ok bool - item interface{} - hash int - ) - if item, ok = s.hashMap.Load(tableID); !ok { - // found new tableID - s.tableBuffers = append(s.tableBuffers, newTableBuffer(tableID)) - hash = len(s.tableBuffers) - 1 - s.hashMap.Store(tableID, hash) - } else { - hash = item.(int) - } - select { - case <-ctx.Done(): - return ctx.Err() - case s.tableBuffers[hash].dataCh <- row: - s.tableBuffers[hash].sendSize.Add(row.ApproximateSize) - if s.tableBuffers[hash].sendSize.Load() > maxNotifySize { - // trigger flush when a table has maxNotifySize - shouldFlush = true - } - s.tableBuffers[hash].sendEvents.Inc() - } - } - if shouldFlush { - // should not block here - select { - case s.notifyChan <- struct{}{}: - default: - } - } - return nil + return s.emitRowChangedEvents(ctx, newTableBuffer, rows...) } func (s *s3Sink) flushLogMeta(ctx context.Context) error { @@ -241,40 +220,12 @@ func (s *s3Sink) flushLogMeta(ctx context.Context) error { return cerror.WrapError(cerror.ErrS3SinkWriteStorage, s.storage.Write(ctx, logMetaFile, data)) } -func (s *s3Sink) flushTableBuffers(ctx context.Context) error { - // TODO use a fixed worker pool - eg, ectx := errgroup.WithContext(ctx) - for _, tb := range s.tableBuffers { - if tb.IsEmpty() { - continue - } - tbReplica := tb - eg.Go(func() error { - log.Info("[FlushRowChangedEvents] flush specify row changed event", - zap.Int64("table", tbReplica.tableID), - zap.Int64("event size", tbReplica.sendEvents.Load())) - return tbReplica.flush(ectx, s) - }) - } - return eg.Wait() -} - func (s *s3Sink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { // we should flush all events before resolvedTs, there are two kind of flush policy // 1. flush row events to a s3 chunk: if the event size is not enough, // TODO: when cdc crashed, we should repair these chunks to a complete file // 2. flush row events to a complete s3 file: if the event size is enough - select { - case <-ctx.Done(): - return 0, ctx.Err() - case <-s.notifyChan: - return resolvedTs, s.flushTableBuffers(ctx) - - case <-time.After(defaultFlushRowChangedEventDuration): - // cannot accumulate enough row events in 10 second - // flush all tables' row events to s3 - return resolvedTs, s.flushTableBuffers(ctx) - } + return s.flushRowChangedEvents(ctx, resolvedTs) } // EmitCheckpointTs update the global resolved ts in log meta @@ -390,7 +341,7 @@ func (s *s3Sink) Close() error { } // NewS3Sink creates new sink support log data to s3 directly -func NewS3Sink(sinkURI *url.URL) (*s3Sink, error) { +func NewS3Sink(ctx context.Context, sinkURI *url.URL, errCh chan error) (*s3Sink, error) { if len(sinkURI.Host) == 0 { return nil, errors.Errorf("please specify the bucket for s3 in %s", sinkURI) } @@ -409,19 +360,23 @@ func NewS3Sink(sinkURI *url.URL) (*s3Sink, error) { return nil, cerror.WrapError(cerror.ErrS3SinkInitialzie, err) } - notifyChan := make(chan struct{}) - tableBuffers := make([]*tableBuffer, 0) - return &s3Sink{ + s := &s3Sink{ prefix: prefix, storage: s3storage, logMeta: newLogMeta(), - encoder: func() codec.EventBatchEncoder { - ret := codec.NewJSONEventBatchEncoder() - ret.(*codec.JSONEventBatchEncoder).SetMixedBuildSupport(true) - return ret - }, + logSink: newLogSink("", s3storage), + } + + // important! we should flush asynchronously in another goroutine + go func() { + if err := s.startFlush(ctx); err != nil && errors.Cause(err) != context.Canceled { + select { + case <-ctx.Done(): + return + case errCh <- err: + } + } + }() - tableBuffers: tableBuffers, - notifyChan: notifyChan, - }, nil + return s, nil } diff --git a/cdc/sink/cdclog/utils.go b/cdc/sink/cdclog/utils.go index e79cf41013b..53328e79e96 100644 --- a/cdc/sink/cdclog/utils.go +++ b/cdc/sink/cdclog/utils.go @@ -14,13 +14,20 @@ package cdclog import ( + "context" "encoding/json" "fmt" + "sync" + "time" + "github.com/pingcap/br/pkg/storage" "github.com/pingcap/log" + "github.com/uber-go/atomic" "go.uber.org/zap" + "golang.org/x/sync/errgroup" "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/cdc/sink/codec" ) const ( @@ -33,6 +40,169 @@ const ( maxUint64 = ^uint64(0) ) +type logUnit interface { + TableID() int64 + Events() *atomic.Int64 + Size() *atomic.Int64 + + dataChan() chan *model.RowChangedEvent + + isEmpty() bool + shouldFlush() bool + // flush data to storage. + flush(ctx context.Context, sink *logSink) error +} + +type logSink struct { + notifyChan chan []logUnit + notifyWaitChan chan struct{} + + encoder func() codec.EventBatchEncoder + units []logUnit + + // file sink use + rootPath string + // s3 sink use + storagePath storage.ExternalStorage + + hashMap sync.Map +} + +func newLogSink(root string, storage storage.ExternalStorage) *logSink { + return &logSink{ + notifyChan: make(chan []logUnit), + notifyWaitChan: make(chan struct{}), + encoder: func() codec.EventBatchEncoder { + ret := codec.NewJSONEventBatchEncoder() + ret.(*codec.JSONEventBatchEncoder).SetMixedBuildSupport(true) + return ret + }, + units: make([]logUnit, 0), + rootPath: root, + storagePath: storage, + } +} + +// s3Sink need this +func (l *logSink) storage() storage.ExternalStorage { + return l.storagePath +} + +// fileSink need this +func (l *logSink) root() string { + return l.rootPath +} + +func (l *logSink) startFlush(ctx context.Context) error { + ticker := time.NewTicker(500 * time.Millisecond) + defer ticker.Stop() + for { + select { + case <-ctx.Done(): + log.Info("[startFlush] log sink stopped") + return ctx.Err() + case needFlushedUnits := <-l.notifyChan: + // try specify buffers + eg, ectx := errgroup.WithContext(ctx) + for _, u := range needFlushedUnits { + uReplica := u + eg.Go(func() error { + log.Info("start Flush asynchronously to storage by caller", + zap.Int64("table id", u.TableID()), + zap.Int64("size", u.Size().Load()), + zap.Int64("event count", u.Events().Load()), + ) + return uReplica.flush(ectx, l) + }) + } + if err := eg.Wait(); err != nil { + return err + } + // tell flush goroutine this time flush finished + l.notifyWaitChan <- struct{}{} + + case <-ticker.C: + // try all tableBuffers + eg, ectx := errgroup.WithContext(ctx) + for _, u := range l.units { + uReplica := u + if u.shouldFlush() { + eg.Go(func() error { + log.Info("start Flush asynchronously to storage", + zap.Int64("table id", u.TableID()), + zap.Int64("size", u.Size().Load()), + zap.Int64("event count", u.Events().Load()), + ) + return uReplica.flush(ectx, l) + }) + } + } + if err := eg.Wait(); err != nil { + return err + } + } + } +} + +func (l *logSink) emitRowChangedEvents(ctx context.Context, newUnit func(int64) logUnit, rows ...*model.RowChangedEvent) error { + for _, row := range rows { + // dispatch row event by tableID + tableID := row.Table.GetTableID() + var ( + ok bool + item interface{} + hash int + ) + if item, ok = l.hashMap.Load(tableID); !ok { + // found new tableID + l.units = append(l.units, newUnit(tableID)) + hash = len(l.units) - 1 + l.hashMap.Store(tableID, hash) + } else { + hash = item.(int) + } + select { + case <-ctx.Done(): + return ctx.Err() + case l.units[hash].dataChan() <- row: + l.units[hash].Size().Add(row.ApproximateSize) + l.units[hash].Events().Inc() + } + } + return nil +} + +func (l *logSink) flushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { + // TODO update flush policy with size + select { + case <-ctx.Done(): + return 0, ctx.Err() + + default: + needFlushedUnits := make([]logUnit, 0, len(l.units)) + for _, u := range l.units { + if !u.isEmpty() { + needFlushedUnits = append(needFlushedUnits, u) + } + } + if len(needFlushedUnits) > 0 { + select { + case <-ctx.Done(): + return 0, ctx.Err() + + case <-time.After(defaultFlushRowChangedEventDuration): + // cannot accumulate enough row events in 5 second + // call flushed worker to flush + l.notifyChan <- needFlushedUnits + // wait flush worker finished + <-l.notifyWaitChan + } + } + } + return resolvedTs, nil + +} + type logMeta struct { Names map[int64]string `json:"names"` GlobalResolvedTS uint64 `json:"global_resolved_ts"` diff --git a/cdc/sink/codec/avro.go b/cdc/sink/codec/avro.go index fb8e5e29fbe..699087b6886 100644 --- a/cdc/sink/codec/avro.go +++ b/cdc/sink/codec/avro.go @@ -98,12 +98,7 @@ func (a *AvroEventBatchEncoder) AppendRowChangedEvent(e *model.RowChangedEvent) mqMessage.Value = nil } - pkeyCols := make([]*model.Column, 0) - for _, col := range e.Columns { - if col.Flag.IsHandleKey() { - pkeyCols = append(pkeyCols, col) - } - } + pkeyCols := e.HandleKeyColumns() res, err := avroEncode(e.Table, a.keySchemaManager, e.TableInfoVersion, pkeyCols) if err != nil { diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index de2173bc129..37862257225 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -32,7 +32,6 @@ import ( "github.com/pingcap/log" timodel "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" - "github.com/pingcap/parser/terror" "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/cdc/sink/common" "github.com/pingcap/ticdc/pkg/config" @@ -63,6 +62,7 @@ const ( defaultBatchReplaceSize = 20 defaultReadTimeout = "2m" defaultWriteTimeout = "2m" + defaultSafeMode = true ) var ( @@ -274,6 +274,8 @@ type sinkParams struct { batchReplaceSize int readTimeout string writeTimeout string + enableOldValue bool + safeMode bool } func (s *sinkParams) Clone() *sinkParams { @@ -289,6 +291,7 @@ var defaultParams = &sinkParams{ batchReplaceSize: defaultBatchReplaceSize, readTimeout: defaultReadTimeout, writeTimeout: defaultWriteTimeout, + safeMode: defaultSafeMode, } func checkTiDBVariable(ctx context.Context, db *sql.DB, variableName, defaultValue string) (string, error) { @@ -349,7 +352,14 @@ func configureSinkURI( } // newMySQLSink creates a new MySQL sink using schema storage -func newMySQLSink(ctx context.Context, changefeedID model.ChangeFeedID, sinkURI *url.URL, filter *tifilter.Filter, opts map[string]string) (Sink, error) { +func newMySQLSink( + ctx context.Context, + changefeedID model.ChangeFeedID, + sinkURI *url.URL, + filter *tifilter.Filter, + replicaConfig *config.ReplicaConfig, + opts map[string]string, +) (Sink, error) { var db *sql.DB params := defaultParams.Clone() @@ -430,6 +440,18 @@ func newMySQLSink(ctx context.Context, changefeedID model.ChangeFeedID, sinkURI params.batchReplaceSize = size } + // TODO: force safe mode in startup phase + s = sinkURI.Query().Get("safe-mode") + if s != "" { + safeModeEnabled, err := strconv.ParseBool(s) + if err != nil { + return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) + } + params.safeMode = safeModeEnabled + } + + params.enableOldValue = replicaConfig.EnableOldValue + // dsn format of the driver: // [username[:password]@][protocol[(address)]]/dbname[?param1=value1&...¶mN=valueN] username := sinkURI.User.Username() @@ -778,19 +800,40 @@ func (s *mysqlSink) prepareDMLs(rows []*model.RowChangedEvent, replicaID uint64, values := make([][]interface{}, 0, len(rows)) replaces := make(map[string][][]interface{}) rowCount := 0 + translateToInsert := s.params.enableOldValue && !s.params.safeMode + + // flush cached batch replace or insert, to keep the sequence of DMLs + flushCacheDMLs := func() { + if s.params.batchReplaceEnabled && len(replaces) > 0 { + replaceSqls, replaceValues := reduceReplace(replaces, s.params.batchReplaceSize) + sqls = append(sqls, replaceSqls...) + values = append(values, replaceValues...) + replaces = make(map[string][][]interface{}) + } + } + for _, row := range rows { var query string var args []interface{} quoteTable := quotes.QuoteSchema(row.Table.Schema, row.Table.Table) - // TODO(leoppro): using `UPDATE` instead of `REPLACE` if the old value is enabled - if len(row.PreColumns) != 0 { - // flush cached batch replace, we must keep the sequence of DMLs - if s.params.batchReplaceEnabled && len(replaces) > 0 { - replaceSqls, replaceValues := reduceReplace(replaces, s.params.batchReplaceSize) - sqls = append(sqls, replaceSqls...) - values = append(values, replaceValues...) - replaces = make(map[string][][]interface{}) + + // Translate to UPDATE if old value is enabled, not in safe mode and is update event + if translateToInsert && len(row.PreColumns) != 0 && len(row.Columns) != 0 { + flushCacheDMLs() + query, args = prepareUpdate(quoteTable, row.PreColumns, row.Columns) + if query != "" { + sqls = append(sqls, query) + values = append(values, args) + rowCount++ } + continue + } + + // Case for delete event or update event + // If old value is enabled and not in safe mode, + // update will be translated to DELETE + INSERT(or REPLACE) SQL. + if len(row.PreColumns) != 0 { + flushCacheDMLs() query, args = prepareDelete(quoteTable, row.PreColumns) if query != "" { sqls = append(sqls, query) @@ -798,9 +841,11 @@ func (s *mysqlSink) prepareDMLs(rows []*model.RowChangedEvent, replicaID uint64, rowCount++ } } + + // Case for insert event or update event if len(row.Columns) != 0 { if s.params.batchReplaceEnabled { - query, args = prepareReplace(quoteTable, row.Columns, false /* appendPlaceHolder */) + query, args = prepareReplace(quoteTable, row.Columns, false /* appendPlaceHolder */, translateToInsert) if query != "" { if _, ok := replaces[query]; !ok { replaces[query] = make([][]interface{}, 0) @@ -809,7 +854,7 @@ func (s *mysqlSink) prepareDMLs(rows []*model.RowChangedEvent, replicaID uint64, rowCount++ } } else { - query, args = prepareReplace(quoteTable, row.Columns, true /* appendPlaceHolder */) + query, args = prepareReplace(quoteTable, row.Columns, true /* appendPlaceHolder */, translateToInsert) sqls = append(sqls, query) values = append(values, args) if query != "" { @@ -820,11 +865,8 @@ func (s *mysqlSink) prepareDMLs(rows []*model.RowChangedEvent, replicaID uint64, } } } - if s.params.batchReplaceEnabled { - replaceSqls, replaceValues := reduceReplace(replaces, s.params.batchReplaceSize) - sqls = append(sqls, replaceSqls...) - values = append(values, replaceValues...) - } + flushCacheDMLs() + dmls := &preparedDMLs{ sqls: sqls, values: values, @@ -864,7 +906,12 @@ func (s *mysqlSink) execDMLs(ctx context.Context, rows []*model.RowChangedEvent, return nil } -func prepareReplace(quoteTable string, cols []*model.Column, appendPlaceHolder bool) (string, []interface{}) { +func prepareReplace( + quoteTable string, + cols []*model.Column, + appendPlaceHolder bool, + translateToInsert bool, +) (string, []interface{}) { var builder strings.Builder columnNames := make([]string, 0, len(cols)) args := make([]interface{}, 0, len(cols)) @@ -880,7 +927,11 @@ func prepareReplace(quoteTable string, cols []*model.Column, appendPlaceHolder b } colList := "(" + buildColumnList(columnNames) + ")" - builder.WriteString("REPLACE INTO " + quoteTable + colList + " VALUES ") + if translateToInsert { + builder.WriteString("INSERT INTO " + quoteTable + colList + " VALUES ") + } else { + builder.WriteString("REPLACE INTO " + quoteTable + colList + " VALUES ") + } if appendPlaceHolder { builder.WriteString("(" + model.HolderString(len(columnNames)) + ");") } @@ -926,6 +977,51 @@ func reduceReplace(replaces map[string][][]interface{}, batchSize int) ([]string return sqls, args } +func prepareUpdate(quoteTable string, preCols, cols []*model.Column) (string, []interface{}) { + var builder strings.Builder + builder.WriteString("UPDATE " + quoteTable + " SET ") + + columnNames := make([]string, 0, len(cols)) + args := make([]interface{}, 0, len(cols)+len(preCols)) + for _, col := range cols { + if col == nil || col.Flag.IsGeneratedColumn() { + continue + } + columnNames = append(columnNames, col.Name) + args = append(args, col.Value) + } + if len(args) == 0 { + return "", nil + } + for i, column := range columnNames { + if i == len(columnNames)-1 { + builder.WriteString("`" + model.EscapeName(column) + "`=?") + } else { + builder.WriteString("`" + model.EscapeName(column) + "`=?,") + } + } + + builder.WriteString(" WHERE ") + colNames, wargs := whereSlice(preCols) + if len(wargs) == 0 { + return "", nil + } + for i := 0; i < len(colNames); i++ { + if i > 0 { + builder.WriteString(" AND ") + } + if wargs[i] == nil { + builder.WriteString(quotes.QuoteName(colNames[i]) + " IS NULL") + } else { + builder.WriteString(quotes.QuoteName(colNames[i]) + "=?") + args = append(args, wargs[i]) + } + } + builder.WriteString(" LIMIT 1;") + sql := builder.String() + return sql, args +} + func prepareDelete(quoteTable string, cols []*model.Column) (string, []interface{}) { var builder strings.Builder builder.WriteString("DELETE FROM " + quoteTable + " WHERE ") @@ -984,13 +1080,13 @@ func isIgnorableDDLError(err error) bool { } } -func getSQLErrCode(err error) (terror.ErrCode, bool) { +func getSQLErrCode(err error) (errors.ErrCode, bool) { mysqlErr, ok := errors.Cause(err).(*dmysql.MySQLError) if !ok { return -1, false } - return terror.ErrCode(mysqlErr.Number), true + return errors.ErrCode(mysqlErr.Number), true } func buildColumnList(names []string) string { diff --git a/cdc/sink/mysql_test.go b/cdc/sink/mysql_test.go index 363b257ce9f..f33fb5c7f97 100644 --- a/cdc/sink/mysql_test.go +++ b/cdc/sink/mysql_test.go @@ -445,6 +445,57 @@ func (s MySQLSinkSuite) TestPrepareDML(c *check.C) { } } +func (s MySQLSinkSuite) TestPrepareUpdate(c *check.C) { + testCases := []struct { + quoteTable string + preCols []*model.Column + cols []*model.Column + expectedSQL string + expectedArgs []interface{} + }{ + { + quoteTable: "`test`.`t1`", + preCols: []*model.Column{}, + cols: []*model.Column{}, + expectedSQL: "", + expectedArgs: nil, + }, + { + quoteTable: "`test`.`t1`", + preCols: []*model.Column{ + {Name: "a", Type: mysql.TypeLong, Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, Value: 1}, + {Name: "b", Type: mysql.TypeVarchar, Flag: 0, Value: "test"}, + }, + cols: []*model.Column{ + {Name: "a", Type: mysql.TypeLong, Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, Value: 1}, + {Name: "b", Type: mysql.TypeVarchar, Flag: 0, Value: "test2"}, + }, + expectedSQL: "UPDATE `test`.`t1` SET `a`=?,`b`=? WHERE `a`=? LIMIT 1;", + expectedArgs: []interface{}{1, "test2", 1}, + }, + { + quoteTable: "`test`.`t1`", + preCols: []*model.Column{ + {Name: "a", Type: mysql.TypeLong, Flag: model.MultipleKeyFlag | model.HandleKeyFlag, Value: 1}, + {Name: "b", Type: mysql.TypeVarString, Flag: model.MultipleKeyFlag | model.HandleKeyFlag, Value: "test"}, + {Name: "c", Type: mysql.TypeLong, Flag: model.GeneratedColumnFlag, Value: 100}, + }, + cols: []*model.Column{ + {Name: "a", Type: mysql.TypeLong, Flag: model.MultipleKeyFlag | model.HandleKeyFlag, Value: 2}, + {Name: "b", Type: mysql.TypeVarString, Flag: model.MultipleKeyFlag | model.HandleKeyFlag, Value: "test2"}, + {Name: "c", Type: mysql.TypeLong, Flag: model.GeneratedColumnFlag, Value: 100}, + }, + expectedSQL: "UPDATE `test`.`t1` SET `a`=?,`b`=? WHERE `a`=? AND `b`=? LIMIT 1;", + expectedArgs: []interface{}{2, "test2", 1, "test"}, + }, + } + for _, tc := range testCases { + query, args := prepareUpdate(tc.quoteTable, tc.preCols, tc.cols) + c.Assert(query, check.Equals, tc.expectedSQL) + c.Assert(args, check.DeepEquals, tc.expectedArgs) + } +} + func (s MySQLSinkSuite) TestMapReplace(c *check.C) { testCases := []struct { quoteTable string @@ -478,7 +529,7 @@ func (s MySQLSinkSuite) TestMapReplace(c *check.C) { for _, tc := range testCases { // multiple times to verify the stability of column sequence in query string for i := 0; i < 10; i++ { - query, args := prepareReplace(tc.quoteTable, tc.cols, false) + query, args := prepareReplace(tc.quoteTable, tc.cols, false, false) c.Assert(query, check.Equals, tc.expectedQuery) c.Assert(args, check.DeepEquals, tc.expectedArgs) } @@ -614,6 +665,7 @@ func (s MySQLSinkSuite) TestSinkParamsClone(c *check.C) { batchReplaceSize: defaultBatchReplaceSize, readTimeout: defaultReadTimeout, writeTimeout: defaultWriteTimeout, + safeMode: defaultSafeMode, }) c.Assert(param2, check.DeepEquals, &sinkParams{ changefeedID: "123", @@ -624,6 +676,7 @@ func (s MySQLSinkSuite) TestSinkParamsClone(c *check.C) { batchReplaceSize: defaultBatchReplaceSize, readTimeout: defaultReadTimeout, writeTimeout: defaultWriteTimeout, + safeMode: defaultSafeMode, }) } diff --git a/cdc/sink/producer/kafka/kafka.go b/cdc/sink/producer/kafka/kafka.go index fe0dc5c2747..9e0f46a6f3f 100644 --- a/cdc/sink/producer/kafka/kafka.go +++ b/cdc/sink/producer/kafka/kafka.go @@ -240,6 +240,12 @@ func (k *kafkaSaramaProducer) run(ctx context.Context) error { atomic.StoreUint64(&k.partitionOffset[msg.Partition].flushed, flushedOffset) k.flushedNotifier.Notify() case err := <-k.asyncClient.Errors(): + // We should not wrap a nil pointer if the pointer is of a subtype of `error` + // because Go would store the type info and the resulted `error` variable would not be nil, + // which will cause the pkg/error library to malfunction. + if err == nil { + return nil + } return cerror.WrapError(cerror.ErrKafkaAsyncSendMessage, err) } } diff --git a/cdc/sink/sink.go b/cdc/sink/sink.go index a115bc46471..942fc5e840d 100644 --- a/cdc/sink/sink.go +++ b/cdc/sink/sink.go @@ -66,15 +66,15 @@ func NewSink(ctx context.Context, changefeedID model.ChangeFeedID, sinkURIStr st case "blackhole": return newBlackHoleSink(ctx, opts), nil case "mysql", "tidb", "mysql+ssl", "tidb+ssl": - return newMySQLSink(ctx, changefeedID, sinkURI, filter, opts) + return newMySQLSink(ctx, changefeedID, sinkURI, filter, config, opts) case "kafka", "kafka+ssl": return newKafkaSaramaSink(ctx, sinkURI, filter, config, opts, errCh) case "pulsar", "pulsar+ssl": return newPulsarSink(ctx, sinkURI, filter, config, opts, errCh) case "local": - return cdclog.NewLocalFileSink(sinkURI) + return cdclog.NewLocalFileSink(ctx, sinkURI, errCh) case "s3": - return cdclog.NewS3Sink(sinkURI) + return cdclog.NewS3Sink(ctx, sinkURI, errCh) default: return nil, cerror.ErrSinkURIInvalid.GenWithStack("the sink scheme (%s) is not supported", sinkURI.Scheme) } diff --git a/cmd/client.go b/cmd/client.go index fb41665cc26..9705c92af23 100644 --- a/cmd/client.go +++ b/cmd/client.go @@ -29,7 +29,7 @@ import ( "github.com/pingcap/ticdc/cdc/kv" "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/pkg/logutil" - "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/ticdc/pkg/version" "github.com/spf13/cobra" pd "github.com/tikv/pd/client" "go.etcd.io/etcd/clientv3" @@ -186,7 +186,7 @@ func newCliCommand() *cobra.Command { } ctx := defaultContext errorTiKVIncompatible := true // Error if TiKV is incompatible. - err = util.CheckClusterVersion(ctx, pdCli, pdEndpoints[0], credential, errorTiKVIncompatible) + err = version.CheckClusterVersion(ctx, pdCli, pdEndpoints[0], credential, errorTiKVIncompatible) if err != nil { return err } diff --git a/cmd/server.go b/cmd/server.go index 1bd61970099..d192dc30a41 100644 --- a/cmd/server.go +++ b/cmd/server.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/ticdc/cdc" "github.com/pingcap/ticdc/pkg/logutil" "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/ticdc/pkg/version" "github.com/spf13/cobra" "go.uber.org/zap" ) @@ -71,7 +72,7 @@ func runEServer(cmd *cobra.Command, args []string) error { return errors.Annotate(err, "can not load timezone, Please specify the time zone through environment variable `TZ` or command line parameters `--tz`") } - util.LogVersionInfo() + version.LogVersionInfo() opts := []cdc.ServerOption{ cdc.PDEndpoints(serverPdAddr), cdc.Address(address), diff --git a/cmd/version.go b/cmd/version.go index 47748e77093..f257e054f8a 100644 --- a/cmd/version.go +++ b/cmd/version.go @@ -16,7 +16,7 @@ package cmd import ( "fmt" - "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/ticdc/pkg/version" "github.com/spf13/cobra" ) @@ -28,6 +28,6 @@ var versionCmd = &cobra.Command{ Use: "version", Short: "Output version information", Run: func(cmd *cobra.Command, args []string) { - fmt.Println(util.GetRawInfo()) + fmt.Println(version.GetRawInfo()) }, } diff --git a/go.mod b/go.mod index 6e5a8b53585..02ae7da5661 100644 --- a/go.mod +++ b/go.mod @@ -25,12 +25,12 @@ require ( github.com/mattn/go-shellwords v1.0.3 github.com/pingcap/br v0.0.0-20200907090854-8a4cd9e0abd1 github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712 - github.com/pingcap/errors v0.11.5-0.20200902104258-eba4f1d8f6de + github.com/pingcap/errors v0.11.5-0.20200917111840-a15ef68f753d github.com/pingcap/failpoint v0.0.0-20200702092429-9f69995143ce github.com/pingcap/kvproto v0.0.0-20200909045102-2ac90648531b github.com/pingcap/log v0.0.0-20200828042413-fce0951f1463 - github.com/pingcap/parser v0.0.0-20200911054040-258297116c4b - github.com/pingcap/tidb v1.1.0-beta.0.20200911063238-51d365fc45fd + github.com/pingcap/parser v0.0.0-20200921063432-e220cfcfd026 + github.com/pingcap/tidb v1.1.0-beta.0.20200921080130-30cfb6af225c github.com/pingcap/tidb-tools v4.0.6-0.20200828085514-03575b185007+incompatible github.com/prometheus/client_golang v1.5.1 github.com/r3labs/diff v1.1.0 @@ -50,3 +50,5 @@ require ( gopkg.in/mgo.v2 v2.0.0-20190816093944-a6b53ec6cb22 // indirect upper.io/db.v3 v3.7.1+incompatible ) + +replace github.com/pingcap/tidb-tools => github.com/pingcap/tidb-tools v4.0.0-beta.0.20200921090810-52b9534ff3d5+incompatible diff --git a/go.sum b/go.sum index c0ab7629289..c3cffcccb90 100644 --- a/go.sum +++ b/go.sum @@ -128,6 +128,8 @@ github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSs github.com/denisenkom/go-mssqldb v0.0.0-20191124224453-732737034ffd/go.mod h1:xbL0rPBG9cCiLr28tMa8zpbdarY27NDyej4t/EjAShU= github.com/dgraph-io/ristretto v0.0.1 h1:cJwdnj42uV8Jg4+KLrYovLiCgIfz9wtWm6E6KA+1tLs= github.com/dgraph-io/ristretto v0.0.1/go.mod h1:T40EBc7CJke8TkpiYfGGKAeFjSaxuFXhuXRyumBd6RE= +github.com/dgraph-io/ristretto v0.0.2 h1:a5WaUrDa0qm0YrAAS1tUykT5El3kt62KNZZeMxQn3po= +github.com/dgraph-io/ristretto v0.0.2/go.mod h1:KPxhHT9ZxKefz+PCeOGsrHpl1qZ7i70dGTu2u+Ahh6E= github.com/dgrijalva/jwt-go v3.2.0+incompatible h1:7qlOGliEKZXTDg6OTjfoBKDXWrumCAMpl/TFQ4/5kLM= github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 h1:tdlZCpZ/P9DhczCTSixgIKmwPv6+wP5DGjqLYw5SUiA= @@ -460,6 +462,8 @@ github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTw github.com/pingcap/errors v0.11.5-0.20190809092503-95897b64e011/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.5-0.20200902104258-eba4f1d8f6de h1:mW8hC2yXTpflfyTeJgcN4aJQfwcYODde8YgjBgAy6do= github.com/pingcap/errors v0.11.5-0.20200902104258-eba4f1d8f6de/go.mod h1:g4vx//d6VakjJ0mk7iLBlKA8LFavV/sAVINT/1PFxeQ= +github.com/pingcap/errors v0.11.5-0.20200917111840-a15ef68f753d h1:TH18wFO5Nq/zUQuWu9ms2urgZnLP69XJYiI2JZAkUGc= +github.com/pingcap/errors v0.11.5-0.20200917111840-a15ef68f753d/go.mod h1:g4vx//d6VakjJ0mk7iLBlKA8LFavV/sAVINT/1PFxeQ= github.com/pingcap/failpoint v0.0.0-20191029060244-12f4ac2fd11d h1:F8vp38kTAckN+v8Jlc98uMBvKIzr1a+UhnLyVYn8Q5Q= github.com/pingcap/failpoint v0.0.0-20191029060244-12f4ac2fd11d/go.mod h1:DNS3Qg7bEDhU6EXNHF+XSv/PGznQaMJ5FWvctpm6pQI= github.com/pingcap/failpoint v0.0.0-20200210140405-f8f9fb234798/go.mod h1:DNS3Qg7bEDhU6EXNHF+XSv/PGznQaMJ5FWvctpm6pQI= @@ -497,8 +501,9 @@ github.com/pingcap/parser v0.0.0-20200623164729-3a18f1e5dceb/go.mod h1:vQdbJqobJ github.com/pingcap/parser v0.0.0-20200803072748-fdf66528323d h1:QQMAWm/b/8EyCrqqcjdO4DcACS06tx8IhKGWC4PTqiQ= github.com/pingcap/parser v0.0.0-20200803072748-fdf66528323d/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= github.com/pingcap/parser v0.0.0-20200901062802-475ea5e2e0a7/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= -github.com/pingcap/parser v0.0.0-20200911054040-258297116c4b h1:olNvO8UWo7Y+t2oWwB46cDj5pyqosgiQts5t8tZlbSc= -github.com/pingcap/parser v0.0.0-20200911054040-258297116c4b/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= +github.com/pingcap/parser v0.0.0-20200921032640-d08ba79f0941/go.mod h1:dMMvhqeowLnAsDWspyalgxXoRUnP09cZ7wAnpt2e/S8= +github.com/pingcap/parser v0.0.0-20200921063432-e220cfcfd026 h1:i+r4P7hb4KpW74nPn+P/hqtsW3fu4U9A4JGAYKWMvtw= +github.com/pingcap/parser v0.0.0-20200921063432-e220cfcfd026/go.mod h1:dMMvhqeowLnAsDWspyalgxXoRUnP09cZ7wAnpt2e/S8= github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2/go.mod h1:s+utZtXDznOiL24VK0qGmtoHjjXNsscJx3m1n8cC56s= github.com/pingcap/pd/v4 v4.0.0-rc.2.0.20200520083007-2c251bd8f181/go.mod h1:q4HTx/bA8aKBa4S7L+SQKHvjRPXCRV0tA0yRw0qkZSA= github.com/pingcap/pd/v4 v4.0.5-0.20200817114353-e465cafe8a91 h1:zCOWP+kIzM6ZsXdu2QoM/W6+3vFZj04MYboMP2Obc0E= @@ -514,15 +519,10 @@ github.com/pingcap/tidb v1.1.0-beta.0.20200715100003-b4da443a3c4c/go.mod h1:TplK github.com/pingcap/tidb v1.1.0-beta.0.20200716023258-b10faca6ff89/go.mod h1:hDlQ5BJ4rLLCOUlvXqW3skyYEjyymzeTA3eXpNEDx38= github.com/pingcap/tidb v1.1.0-beta.0.20200820092836-c5b7658b0896 h1:l2UJF9cFxwaMMNMjguqrfiC7sFZrEqbtEmAAWFyHx9w= github.com/pingcap/tidb v1.1.0-beta.0.20200820092836-c5b7658b0896/go.mod h1:IAStISSVhEI9Gp/sE4w6Ms0WxpdBJ9qNTczNyskvd5A= -github.com/pingcap/tidb v1.1.0-beta.0.20200911063238-51d365fc45fd h1:NzlXvchm6aPuW29ciy63vUwznkp4OrQVnF6/TTdGcRg= -github.com/pingcap/tidb v1.1.0-beta.0.20200911063238-51d365fc45fd/go.mod h1:hZoU6jeZIj9jViblw0Pf1aOBJajgI82eMqlC7HYtRWI= -github.com/pingcap/tidb-tools v4.0.0-beta.1.0.20200306084441-875bd09aa3d5+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200421113014-507d2bb3a15e+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200514040632-f76b3e428e19+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tidb-tools v4.0.0-rc.2.0.20200521050818-6dd445d83fe0+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tidb-tools v4.0.1-0.20200530144555-cdec43635625+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tidb-tools v4.0.6-0.20200828085514-03575b185007+incompatible h1:1GY6Qu5pT7JZ4QwkPcz+daXKhkDgKY1F6qKxifSp+tI= -github.com/pingcap/tidb-tools v4.0.6-0.20200828085514-03575b185007+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= +github.com/pingcap/tidb v1.1.0-beta.0.20200921080130-30cfb6af225c h1:1N3hX0obP2XsTLozvrou05W8Pl2R9BpQgePYPtJtXWc= +github.com/pingcap/tidb v1.1.0-beta.0.20200921080130-30cfb6af225c/go.mod h1:eZL1RbU3Ct+4ryN8cM18y7zcO2FEvq/wSZFgIR2H6L0= +github.com/pingcap/tidb-tools v4.0.0-beta.0.20200921090810-52b9534ff3d5+incompatible h1:G4YTSvg0MfstbN52K0V6jEeUdSm/6D42uzZu3XQirss= +github.com/pingcap/tidb-tools v4.0.0-beta.0.20200921090810-52b9534ff3d5+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= github.com/pingcap/tipb v0.0.0-20200417094153-7316d94df1ee/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= github.com/pingcap/tipb v0.0.0-20200522051215-f31a15d98fce/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= @@ -693,6 +693,8 @@ go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.6.0 h1:Ezj3JGmsOnG1MoRWQkPBsKLe9DwWD9QeXzTRzzldNVk= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= +go.uber.org/atomic v1.7.0 h1:ADUqmZGgLDDfbSL9ZmPxKTybcoEYHgpYfELNoN+7hsw= +go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/automaxprocs v1.2.0/go.mod h1:YfO3fm683kQpzETxlTGZhGIVmXAhaw3gxeBADbpZtnU= go.uber.org/dig v1.8.0/go.mod h1:X34SnWGr8Fyla9zQNO2GSO2D+TIuqB14OS8JhYocIyw= go.uber.org/fx v1.10.0/go.mod h1:vLRicqpG/qQEzno4SYU86iCwfT95EZza+Eba0ItuxqY= @@ -703,6 +705,8 @@ go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+ go.uber.org/multierr v1.4.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= go.uber.org/multierr v1.5.0 h1:KCa4XfM8CWFCpxXRGok+Q0SS/0XBhMDbHHGABQLvD2A= go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= +go.uber.org/multierr v1.6.0 h1:y6IPFStTAIT5Ytl7/XYmHvzXQ7S3g/IeZW9hyZ5thw4= +go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee h1:0mgffUl7nfd+FpvXMVz4IDEaUSmT1ysygQC7qYo7sG4= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.8.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= diff --git a/integration/framework/avro_kafka_docker_env.go b/integration/framework/avro_kafka_docker_env.go index 1c63314a532..0acdac1fc51 100644 --- a/integration/framework/avro_kafka_docker_env.go +++ b/integration/framework/avro_kafka_docker_env.go @@ -149,6 +149,10 @@ func (e *AvroKafkaDockerEnv) RunTest(task Task) { log.Info("Start running task", zap.String("name", task.Name())) err = task.Run(taskCtx) if err != nil { + err1 := e.DumpStdout() + if err1 != nil { + log.Warn("Failed to dump container logs", zap.Error(err1)) + } e.TearDown() log.Fatal("RunTest: task failed", zap.String("name", task.Name()), zap.Error(err)) } diff --git a/integration/framework/docker_compose_op.go b/integration/framework/docker_compose_op.go index e2b8329f154..795230ddad4 100644 --- a/integration/framework/docker_compose_op.go +++ b/integration/framework/docker_compose_op.go @@ -14,9 +14,11 @@ package framework import ( + "os" "os/exec" "time" + "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/ticdc/pkg/retry" "go.uber.org/zap" @@ -59,6 +61,24 @@ func runCmdHandleError(cmd *exec.Cmd) []byte { return bytes } +// DumpStdout dumps all container logs +func (d *dockerComposeOperator) DumpStdout() error { + log.Info("Dumping container logs") + cmd := exec.Command("docker-compose", "-f", d.fileName, "logs", "-t") + f, err := os.Create("./stdout.log") + if err != nil { + return errors.AddStack(err) + } + defer f.Close() + cmd.Stdout = f + err = cmd.Run() + if err != nil { + return errors.AddStack(err) + } + + return nil +} + // TearDown terminates a docker-compose service and remove all volumes func (d *dockerComposeOperator) TearDown() { log.Info("Start tearing down docker-compose services") diff --git a/integration/framework/sql_helper.go b/integration/framework/sql_helper.go index c04f1757274..a8b534c7a5d 100644 --- a/integration/framework/sql_helper.go +++ b/integration/framework/sql_helper.go @@ -249,13 +249,16 @@ func (r *syncSQLRequest) Send() Awaitable { defer rows.Close() if !rows.Next() { - log.Warn("ReadBack:", zap.Error(err)) - return - } - r.data, err = rowsToMap(rows) - if err != nil { - log.Warn("ReadBack", zap.Error(err)) - return + // Upstream does not have the row + if r.requestType != sqlRequestTypeDelete { + log.Warn("ReadBack: no row, likely to be bug") + } + } else { + r.data, err = rowsToMap(rows) + if err != nil { + log.Warn("ReadBack", zap.Error(err)) + return + } } atomic.StoreUint32(&r.hasReadBack, 1) @@ -325,7 +328,7 @@ func (s *sqlRequest) upsert(ctx context.Context) error { } func (s *sqlRequest) delete(ctx context.Context) error { - db, err := sqlbuilder.New("mysql", s.helper.downstream) + db, err := sqlbuilder.New("mysql", s.helper.upstream) if err != nil { return errors.AddStack(err) } @@ -447,7 +450,7 @@ func getUniqueIndexColumn(ctx context.Context, db sqlbuilder.Database, dbName st row, err := db.QueryRowContext(ctx, ` SELECT GROUP_CONCAT(COLUMN_NAME SEPARATOR ' ') FROM INFORMATION_SCHEMA.STATISTICS WHERE TABLE_SCHEMA = ? AND TABLE_NAME = ? - GROUP BY INDEX_NAME + GROUP BY INDEX_NAME ORDER BY FIELD(INDEX_NAME,'PRIMARY') DESC `, dbName, tableName) if err != nil { diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index d527a6d4fc0..f92cfab56c4 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -45,14 +45,63 @@ "list": [ { "builtIn": 1, + "datasource": "-- Grafana --", + "enable": false, + "expr": "", + "hide": true, + "iconColor": "#F2495C", + "limit": 100, + "name": "", + "showIn": 0, + "tagKeys": "", + "textFormat": "", + "titleFormat": "", + "type": "dashboard", + "useValueForTime": false + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "enable": true, + "expr": "max(ticdc_processor_checkpoint_ts_lag) by (changefeed, capture) > BOOL $spike_threshold", + "hide": true, + "iconColor": "#F2495C", + "limit": 100, + "name": "Latency spike", + "showIn": 0, + "tagKeys": "changefeed", + "tags": [], + "titleFormat": "Latency spike", + "type": "tags", + "useValueForTime": false + }, + { "datasource": "${DS_TEST-CLUSTER}", "enable": true, + "expr": "min(up{job=~\"tikv|ticdc\"}) by (job, instance) == BOOL 0", "hide": true, - "iconColor": "rgba(0, 211, 255, 1)", + "iconColor": "#FF9830", + "limit": 100, + "name": "Server down", + "showIn": 0, + "tagKeys": "instance", + "tags": [], + "textFormat": "instance", + "titleFormat": "Down", + "type": "tags" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "enable": false, + "expr": "sum(ALERTS{alertstate=\"firing\", alertname=~\"ticdc.*\"}) by (alertname) > BOOL 0", + "hide": false, + "iconColor": "#B877D9", "limit": 100, - "name": "Annotations & Alerts", + "name": "All TiCDC Alerts", "showIn": 0, - "type": "dashboard" + "tagKeys": "alertname", + "tags": [], + "titleFormat": "Alert Name", + "type": "tags" } ] }, @@ -60,7 +109,7 @@ "gnetId": null, "graphTooltip": 1, "id": null, - "iteration": 1597641819763, + "iteration": 1600073138046, "links": [], "panels": [ { @@ -102,6 +151,7 @@ }, "lines": true, "linewidth": 1, + "links": [], "nullPointMode": "null", "options": { "dataLinks": [] @@ -120,8 +170,15 @@ "expr": "(time() - process_start_time_seconds{job=\"ticdc\"})", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}", + "legendFormat": "TiCDC - {{instance}}", "refId": "A" + }, + { + "expr": "(time() - process_start_time_seconds{job=\"tikv\"})", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "TiKV - {{instance}}", + "refId": "B" } ], "thresholds": [], @@ -616,13 +673,13 @@ }, { "aliasColors": {}, - "bars": false, + "bars": true, "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "global checkpoint ts of processor", - "fill": 1, + "fill": 0, "gridPos": { "h": 5, "w": 8, @@ -643,14 +700,19 @@ "values": true }, "lines": true, - "linewidth": 1, + "linewidth": 2, "links": [], "nullPointMode": "null", "percentage": false, "pointradius": 2, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/approximate current time.*/", + "bars": false + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, @@ -925,7 +987,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_processor_checkpoint_ts_lag{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "expr": "max(ticdc_processor_checkpoint_ts_lag{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", "format": "time_series", "interval": "", "intervalFactor": 1, @@ -1148,6 +1210,114 @@ "yBucketNumber": null, "yBucketSize": null }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 14 + }, + "hiddenSeries": false, + "id": 98, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.95, sum(rate(ticdc_processor_flush_event_duration_seconds_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p95", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.99, sum(rate(ticdc_processor_flush_event_duration_seconds_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p99", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.999, sum(rate(ticdc_processor_flush_event_duration_seconds_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p999", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "flush sink duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 2, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, { "cards": { "cardPadding": 0, @@ -1158,7 +1328,8 @@ "colorScale": "linear", "colorScheme": "interpolateSpectral", "exponent": 0.5, - "min": 0, + "max": null, + "min": 1, "mode": "spectrum" }, "dataFormat": "tsbuckets", @@ -1167,8 +1338,8 @@ "gridPos": { "h": 7, "w": 12, - "x": 12, - "y": 14 + "x": 0, + "y": 21 }, "heatmap": {}, "hideZeroBuckets": true, @@ -1188,6 +1359,8 @@ "values": true }, "links": [], + "repeat": null, + "repeatDirection": "h", "reverseYBuckets": false, "targets": [ { @@ -1224,6 +1397,114 @@ "yBucketNumber": null, "yBucketSize": null }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of events received from kv client event channel", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 21 + }, + "hiddenSeries": false, + "id": 35, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.95, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p95", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p99", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p999", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "sink exec txn duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 2, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, { "aliasColors": {}, "bars": false, @@ -1237,7 +1518,7 @@ "h": 7, "w": 12, "x": 0, - "y": 21 + "y": 28 }, "hiddenSeries": false, "id": 95, @@ -1378,7 +1659,7 @@ "h": 7, "w": 12, "x": 12, - "y": 21 + "y": 28 }, "hiddenSeries": false, "id": 96, @@ -1492,7 +1773,7 @@ "h": 7, "w": 12, "x": 0, - "y": 28 + "y": 35 }, "hiddenSeries": false, "id": 32, @@ -1586,7 +1867,7 @@ "h": 7, "w": 12, "x": 12, - "y": 28 + "y": 35 }, "hiddenSeries": false, "id": 83, @@ -1681,25 +1962,27 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of events received from kv client event channel", - "fill": 1, + "description": "input chan size of table mounter", + "fill": 0, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 35 + "y": 42 }, "hiddenSeries": false, - "id": 35, + "id": 37, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -1721,35 +2004,21 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "expr": "sum(ticdc_mounter_input_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}-p95", + "legendFormat": "{{capture}}", "refId": "A" - }, - { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p99", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_txn_exec_duration_bucket{changefeed=~\"$changefeed\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p999", - "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "exec txn duration", + "title": "mounter input chan size", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -1762,9 +2031,9 @@ }, "yaxes": [ { - "format": "s", + "format": "none", "label": null, - "logBase": 2, + "logBase": 1, "max": null, "min": null, "show": true @@ -1796,7 +2065,7 @@ "h": 7, "w": 12, "x": 12, - "y": 35 + "y": 42 }, "hiddenSeries": false, "id": 5, @@ -1883,17 +2152,17 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "input chan size of table mounter", + "description": "checkpoint channel size for mq sink", "fill": 0, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 42 + "y": 49 }, "hiddenSeries": false, - "id": 37, + "id": 48, "legend": { "alignAsTable": true, "avg": false, @@ -1925,7 +2194,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_mounter_input_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "expr": "sum(ticdc_sink_mq_checkpoint_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", "format": "time_series", "intervalFactor": 1, "legendFormat": "{{capture}}", @@ -1936,7 +2205,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "mounter input chan size", + "title": "sink mq checkpoint chan size", "tooltip": { "shared": true, "sort": 2, @@ -1985,7 +2254,7 @@ "h": 7, "w": 12, "x": 12, - "y": 42 + "y": 49 }, "id": 82, "legend": { @@ -2067,17 +2336,17 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Bucketed histogram of processing time (s) of unmarshal and mount in mounter.", + "description": "Bucketed histogram of processing time (s) of waiting event prepare in processor.", "fill": 0, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 49 + "y": 56 }, "hiddenSeries": false, - "id": 55, + "id": 56, "legend": { "alignAsTable": true, "avg": false, @@ -2109,34 +2378,25 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_mounter_unmarshal_and_mount_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-p99", - "refId": "A" - }, - { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_mounter_unmarshal_and_mount_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", + "expr": "histogram_quantile(0.999, sum(rate(ticdc_processor_wait_event_prepare_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", "format": "time_series", - "hide": true, - "instant": false, "intervalFactor": 1, "legendFormat": "{{capture}}-p999", - "refId": "B" + "refId": "A" }, { - "expr": "rate(ticdc_mounter_unmarshal_and_mount_sum{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) / rate(ticdc_mounter_unmarshal_and_mount_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])", + "expr": "rate(ticdc_processor_wait_event_prepare_sum{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) / rate(ticdc_processor_wait_event_prepare_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])", "format": "time_series", "intervalFactor": 1, "legendFormat": "{{capture}}-avg", - "refId": "C" + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "mounter unmarshal and mount", + "title": "processor wait event prepare", "tooltip": { "shared": true, "sort": 2, @@ -2160,7 +2420,7 @@ "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -2186,7 +2446,7 @@ "h": 7, "w": 12, "x": 12, - "y": 49 + "y": 56 }, "hiddenSeries": false, "id": 34, @@ -2281,17 +2541,17 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "checkpoint channel size for mq sink", + "description": "size of row changed event output channel from table to processor", "fill": 0, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 56 + "y": 63 }, "hiddenSeries": false, - "id": 48, + "id": 38, "legend": { "alignAsTable": true, "avg": false, @@ -2323,7 +2583,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_sink_mq_checkpoint_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "expr": "sum(ticdc_processor_txn_output_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", "format": "time_series", "intervalFactor": 1, "legendFormat": "{{capture}}", @@ -2334,7 +2594,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "sink mq checkpoint chan size", + "title": "processor table output chan size", "tooltip": { "shared": true, "sort": 2, @@ -2384,7 +2644,7 @@ "h": 7, "w": 12, "x": 12, - "y": 56 + "y": 63 }, "hiddenSeries": false, "id": 36, @@ -2479,40 +2739,51 @@ "align": false, "alignLevel": null } - }, + } + ], + "title": "Changefeed", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 2 + }, + "id": 13, + "panels": [ { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Bucketed histogram of processing time (s) of waiting event prepare in processor.", - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 63 + "y": 3 }, "hiddenSeries": false, - "id": 56, + "id": 15, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": true, + "hideEmpty": false, + "max": false, "min": false, "rightSide": true, "show": true, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null", "options": { "dataLinks": [] @@ -2528,28 +2799,21 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_processor_wait_event_prepare_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", + "expr": "ticdc_kvclient_event_feed_count", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}-p999", + "legendFormat": "{{instance}}", "refId": "A" - }, - { - "expr": "rate(ticdc_processor_wait_event_prepare_sum{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) / rate(ticdc_processor_wait_event_prepare_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-avg", - "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "processor wait event prepare", + "title": "Event Feed Count", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -2562,7 +2826,7 @@ }, "yaxes": [ { - "format": "s", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -2589,33 +2853,31 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "size of row changed event output channel from table to processor", - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 63 + "y": 3 }, "hiddenSeries": false, - "id": 38, + "id": 28, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": true, + "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null", "options": { "dataLinks": [] @@ -2631,10 +2893,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_processor_txn_output_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "expr": "sum(rate(ticdc_kvclient_event_feed_error_count[30s])) by (instance, type)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}", + "legendFormat": "{{instance}} - {{type}}", "refId": "A" } ], @@ -2642,10 +2904,10 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "processor table output chan size", + "title": "Event Feed Error Count", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -2658,7 +2920,7 @@ }, "yaxes": [ { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -2678,57 +2940,44 @@ "align": false, "alignLevel": null } - } - ], - "title": "Changefeed", - "type": "row" - }, - { - "collapsed": true, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 2 - }, - "id": 58, - "panels": [ + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, + "decimals": null, "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 71 + "y": 10 }, - "id": 60, + "hiddenSeries": false, + "id": 17, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": true, + "max": false, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null", + "options": { + "dataLinks": [] + }, "paceLength": 10, "percentage": false, - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -2737,19 +2986,18 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_thread_cpu_seconds_total{instance=~\"$tikv_instance\", name=~\"cdc\"}[1m])) by (instance)", + "expr": "histogram_quantile(0.95, sum(rate(ticdc_kvclient_event_size_bytes_bucket[1m])) by (le, instance))", "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{instance}}", - "refId": "A", - "step": 4 + "intervalFactor": 1, + "legendFormat": "{{instance}} 95", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "CDC endpoint CPU", + "title": "Event Size", "tooltip": { "shared": true, "sort": 0, @@ -2765,7 +3013,7 @@ }, "yaxes": [ { - "format": "percentunit", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -2792,26 +3040,26 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 71 + "y": 10 }, - "id": 62, + "hiddenSeries": false, + "id": 29, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": true, + "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -2819,9 +3067,12 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "dataLinks": [] + }, "paceLength": 10, "percentage": false, - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -2830,19 +3081,18 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_thread_cpu_seconds_total{instance=~\"$tikv_instance\", name=~\"cdcwkr.*\"}[1m])) by (instance)", + "expr": "sum(rate(ticdc_kvclient_pull_event_count{changefeed=~\"$changefeed\"}[1m])) by (instance, type)", "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{instance}}", - "refId": "A", - "step": 4 + "intervalFactor": 1, + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "CDC worker CPU", + "title": "EventFeed Receive Event Count By Instance", "tooltip": { "shared": true, "sort": 0, @@ -2858,7 +3108,7 @@ }, "yaxes": [ { - "format": "percentunit", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -2880,141 +3130,111 @@ } }, { - "columns": [ - { - "text": "Current", - "value": "current" - }, - { - "text": "Max", - "value": "max" - } - ], + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "fontSize": "90%", + "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, - "w": 7, + "h": 7, + "w": 12, "x": 0, - "y": 79 + "y": 17 }, - "hideTimeOverride": true, - "id": 64, + "hiddenSeries": false, + "id": 31, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, "links": [], - "pageSize": null, - "scroll": true, - "showHeader": true, - "sort": { - "col": 2, - "desc": false + "nullPointMode": "null", + "options": { + "dataLinks": [] }, - "styles": [ - { - "alias": "Instance", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" - ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "mappingType": 1, - "pattern": "Metric", - "thresholds": [], - "type": "string", - "unit": "short", - "valueMaps": [] - }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ { - "alias": "Time", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" - ], - "dateFormat": "MM-DD HH:mm:ss.SSS", - "decimals": 2, - "mappingType": 1, - "pattern": "Max", - "thresholds": [], - "type": "date", - "unit": "short" + "alias": "/.*batch-resolved/", + "yaxis": 2 } ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, "targets": [ { - "expr": "tikv_cdc_min_resolved_ts{instance=~\"$tikv_instance\"}", + "expr": "sum(rate(ticdc_kvclient_send_event_count{changefeed=~\"$changefeed\"}[1m])) by (capture, type)", "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{instance}}", + "intervalFactor": 1, + "legendFormat": "{{capture}}-{{type}}", "refId": "A" }, { - "expr": "max(pd_cluster_tso) * 1000", + "expr": "sum(rate(ticdc_kvclient_batch_resolved_event_size_count{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture, table)", "format": "time_series", - "hide": true, - "intervalFactor": 2, - "legendFormat": "current time (s)", + "intervalFactor": 1, + "legendFormat": "{{capture}}-batch-resolved", "refId": "B" } ], - "timeFrom": "1s", - "title": "Min resolved ts", - "transform": "timeseries_aggregations", - "type": "table" - }, - { - "columns": [ - { - "text": "Current", - "value": "current" - } - ], - "datasource": "${DS_TEST-CLUSTER}", - "fontSize": "90%", - "gridPos": { - "h": 8, - "w": 5, - "x": 7, - "y": 79 + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Dispatch Event Count By Instance", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" }, - "hideTimeOverride": true, - "id": 66, - "links": [], - "pageSize": null, - "scroll": true, - "showHeader": true, - "sort": { - "col": 0, - "desc": false + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] }, - "styles": [ + "yaxes": [ { - "alias": "Option", - "colorMode": null, - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "mappingType": 2, - "pattern": "Metric", - "preserveFormat": false, - "sanitize": false, - "type": "number" - } - ], - "targets": [ + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, { - "expr": "tikv_cdc_min_resolved_ts_region{instance=~\"$tikv_instance\"}", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{instance}}", - "refId": "A" + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true } ], - "timeFrom": "1s", - "title": "Min resolved Region", - "transform": "timeseries_aggregations", - "type": "table" + "yaxis": { + "align": false, + "alignLevel": null + } }, { "cards": { @@ -3031,17 +3251,17 @@ }, "dataFormat": "tsbuckets", "datasource": "${DS_TEST-CLUSTER}", - "description": "The time consumed to CDC incremental scan", + "description": "", "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 79 + "y": 17 }, "heatmap": {}, "hideZeroBuckets": true, "highlightCards": true, - "id": 68, + "id": 97, "legend": { "alignAsTable": true, "avg": false, @@ -3059,7 +3279,7 @@ "reverseYBuckets": false, "targets": [ { - "expr": "sum(rate(tikv_cdc_scan_duration_seconds_bucket{instance=~\"$tikv_instance\"}[1m])) by (le)", + "expr": "sum(rate(ticdc_kvclient_batch_resolved_event_size_bucket{instance=~\"$tikv_instance\"}[1m])) by (le)", "format": "heatmap", "instant": false, "intervalFactor": 2, @@ -3067,7 +3287,7 @@ "refId": "A" } ], - "title": "Initial scan duration", + "title": "batch resolved count", "tooltip": { "show": true, "showHistogram": true @@ -3081,7 +3301,7 @@ "xBucketSize": null, "yAxis": { "decimals": 1, - "format": "s", + "format": "none", "logBase": 1, "max": null, "min": null, @@ -3098,29 +3318,26 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "", + "description": "Puller entry channel buffer size", "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 87 + "y": 24 }, - "id": 70, + "hiddenSeries": false, + "id": 43, "legend": { "alignAsTable": true, "avg": false, "current": true, "hideEmpty": false, - "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": null, - "sortDesc": null, "total": false, "values": true }, @@ -3128,9 +3345,12 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "dataLinks": [] + }, "paceLength": 10, "percentage": false, - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -3139,10 +3359,10 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99999, sum(rate(tikv_cdc_resolved_ts_gap_seconds_bucket{instance=~\"$tikv_instance\"}[1m])) by (le, instance))", + "expr": "sum(ticdc_puller_output_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}", + "legendFormat": "{{capture}} - {{table}}", "refId": "A" } ], @@ -3150,7 +3370,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "99.9% Resolved ts lag duration", + "title": "puller output chan size", "tooltip": { "shared": true, "sort": 0, @@ -3166,9 +3386,9 @@ }, "yaxes": [ { - "format": "s", + "format": "short", "label": null, - "logBase": 10, + "logBase": 1, "max": null, "min": null, "show": true @@ -3193,29 +3413,26 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "", + "description": "Puller event channel size", "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 87 + "y": 24 }, - "id": 72, + "hiddenSeries": false, + "id": 40, "legend": { "alignAsTable": true, "avg": false, "current": true, "hideEmpty": false, - "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -3223,9 +3440,12 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "dataLinks": [] + }, "paceLength": 10, "percentage": false, - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -3234,10 +3454,10 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(tikv_cdc_scan_duration_seconds_bucket{instance=~\"$tikv_instance\"}[1m])) by (le, instance))", + "expr": "sum(ticdc_puller_event_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}", + "legendFormat": "{{capture}}", "refId": "A" } ], @@ -3245,7 +3465,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "99.9% Scan duration", + "title": "puller event chan size", "tooltip": { "shared": true, "sort": 0, @@ -3261,7 +3481,7 @@ }, "yaxes": [ { - "format": "s", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -3274,7 +3494,7 @@ "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -3288,30 +3508,26 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The memory usage per TiKV instance", - "editable": true, - "error": false, - "fill": 0, - "grid": {}, + "description": "forwarded resolved ts in a single puller", + "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 95 + "y": 31 }, - "id": 74, + "hiddenSeries": false, + "id": 52, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": "max", - "sortDesc": true, "total": false, "values": true }, @@ -3319,9 +3535,12 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "dataLinks": [] + }, "paceLength": 10, "percentage": false, - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -3330,40 +3549,19 @@ "steppedLine": false, "targets": [ { - "expr": "avg(process_resident_memory_bytes{instance=~\"$tikv_instance\", job=\"tikv\"}) by (instance)", + "expr": "sum(ticdc_puller_resolved_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", "format": "time_series", - "hide": true, - "intervalFactor": 2, - "legendFormat": "tikv-{{instance}}", - "refId": "A", - "step": 10 - }, - { - "expr": "avg(process_resident_memory_bytes{instance=~\"$tikv_instance\", job=\"cdc\"}) by (instance)", - "format": "time_series", - "hide": true, - "intervalFactor": 2, - "legendFormat": "cdc-{{instance}}", - "refId": "B", - "step": 10 - }, - { - "expr": "(avg(process_resident_memory_bytes{instance=~\"$tikv_instance\", job=\"tikv\"}) by (instance)) - (avg(tikv_engine_block_cache_size_bytes{instance=~\"$tikv_instance\", db=\"kv\"}) by(instance))", - "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "tikv-{{instance}}", - "refId": "C", - "step": 10 + "intervalFactor": 1, + "legendFormat": "{{capture}}-{{table}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Memory without block cache", + "title": "puller resolved ts", "tooltip": { - "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -3378,11 +3576,11 @@ }, "yaxes": [ { - "format": "bytes", + "format": "dateTimeAsIso", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -3405,30 +3603,26 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The memory usage per TiKV instance", - "editable": true, - "error": false, - "fill": 0, - "grid": {}, + "description": "Puller in memory buffer size", + "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 95 + "y": 31 }, - "id": 76, + "hiddenSeries": false, + "id": 50, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -3436,9 +3630,12 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "dataLinks": [] + }, "paceLength": 10, "percentage": false, - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -3447,22 +3644,19 @@ "steppedLine": false, "targets": [ { - "expr": "avg(tikv_cdc_pending_bytes{instance=~\"$tikv_instance\", job=\"tikv\"}) by (instance)", + "expr": "sum(ticdc_puller_mem_buffer_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "tikv-{{instance}}", - "refId": "A", - "step": 10 + "intervalFactor": 1, + "legendFormat": "{{capture}} - {{table}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "CDC pending bytes in memory", + "title": "puller mem buffer size", "tooltip": { - "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -3477,11 +3671,11 @@ }, "yaxes": [ { - "format": "bytes", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -3504,30 +3698,26 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The memory usage per TiKV instance", - "editable": true, - "error": false, - "fill": 0, - "grid": {}, + "description": "Puller entry sorter resolved channel size", + "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 103 + "y": 38 }, - "id": 78, + "hiddenSeries": false, + "id": 42, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": null, - "sortDesc": null, "total": false, "values": true }, @@ -3535,9 +3725,12 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "dataLinks": [] + }, "paceLength": 10, "percentage": false, - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -3546,22 +3739,19 @@ "steppedLine": false, "targets": [ { - "expr": "avg(tikv_cdc_captured_region_total{instance=~\"$tikv_instance\"}) by (instance)", + "expr": "sum(ticdc_puller_entry_sorter_resolved_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "tikv-{{instance}}", - "refId": "A", - "step": 10 + "intervalFactor": 1, + "legendFormat": "{{capture}}-{{table}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Captured region count", + "title": "entry sorter resolved chan size", "tooltip": { - "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -3576,11 +3766,11 @@ }, "yaxes": [ { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -3596,43 +3786,30 @@ "align": false, "alignLevel": null } - } - ], - "title": "TiKV", - "type": "row" - }, - { - "collapsed": true, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 3 - }, - "id": 13, - "panels": [ + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "Puller entry sorter unsoreted items size", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 2 + "x": 12, + "y": 38 }, "hiddenSeries": false, - "id": 15, + "id": 51, "legend": { "alignAsTable": true, "avg": false, "current": true, "hideEmpty": false, - "max": false, + "max": true, "min": false, "rightSide": true, "show": true, @@ -3641,6 +3818,7 @@ }, "lines": true, "linewidth": 1, + "links": [], "nullPointMode": "null", "options": { "dataLinks": [] @@ -3656,10 +3834,10 @@ "steppedLine": false, "targets": [ { - "expr": "ticdc_kvclient_event_feed_count", + "expr": "sum(ticdc_puller_entry_sorter_unsorted_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}", + "legendFormat": "{{capture}}-{{table}}", "refId": "A" } ], @@ -3667,7 +3845,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Event Feed Count", + "title": "entry sorter unsorted size", "tooltip": { "shared": true, "sort": 0, @@ -3704,37 +3882,115 @@ "alignLevel": null } }, + { + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 45 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 101, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "max(rate(ticdc_mounter_unmarshal_and_mount_bucket{capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "mounter unmarshal and mount duration", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "fill": 1, + "description": "Bucketed histogram of processing time (s) of unmarshal and mount in mounter.", + "fill": 0, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 2 + "y": 45 }, "hiddenSeries": false, - "id": 28, + "id": 55, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": true, - "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, + "links": [], "nullPointMode": "null", "options": { "dataLinks": [] @@ -3750,21 +4006,31 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_kvclient_event_feed_error_count[30s])) by (instance, type)", + "expr": "histogram_quantile(0.99, sum(rate(ticdc_mounter_unmarshal_and_mount_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", "format": "time_series", + "hide": false, "intervalFactor": 1, - "legendFormat": "{{instance}} - {{type}}", + "legendFormat": "{{capture}}-p99", "refId": "A" + }, + { + "expr": "histogram_quantile(0.999, sum(rate(ticdc_mounter_unmarshal_and_mount_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", + "format": "time_series", + "hide": true, + "instant": false, + "intervalFactor": 1, + "legendFormat": "{{capture}}-p999", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Event Feed Error Count", + "title": "mounter unmarshal and mount duration", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -3777,7 +4043,7 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -3785,7 +4051,7 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -3799,27 +4065,104 @@ } }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "decimals": null, - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 9 - }, - "hiddenSeries": false, - "id": 17, + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 52 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 99, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(rate(ticdc_puller_entry_sorter_sort_bucket{capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "entry sorter sort duration", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Bucketed histogram of processing time (s) of merge in entry sorter", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 52 + }, + "hiddenSeries": false, + "id": 53, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "max": true, "min": false, "rightSide": true, "show": true, @@ -3828,6 +4171,7 @@ }, "lines": true, "linewidth": 1, + "links": [], "nullPointMode": "null", "options": { "dataLinks": [] @@ -3843,18 +4187,26 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_kvclient_event_size_bytes_bucket[1m])) by (le, instance))", + "expr": "histogram_quantile(0.99, sum(rate(ticdc_puller_entry_sorter_sort_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,table))", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}} 95", + "legendFormat": "{{table}}-p99", "refId": "A" + }, + { + "expr": "histogram_quantile(0.90, sum(rate(ticdc_puller_entry_sorter_sort_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,table))", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "{{table}}-p90", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Event Size", + "title": "entry sorter sort duration", "tooltip": { "shared": true, "sort": 0, @@ -3870,9 +4222,9 @@ }, "yaxes": [ { - "format": "bytes", + "format": "s", "label": null, - "logBase": 1, + "logBase": 2, "max": null, "min": null, "show": true @@ -3891,28 +4243,104 @@ "alignLevel": null } }, + { + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 59 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 100, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(rate(ticdc_puller_entry_sorter_merge_bucket{capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "entry sorter merge duration", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "Bucketed histogram of processing time (s) of merge in entry sorter.", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 9 + "y": 59 }, "hiddenSeries": false, - "id": 29, + "id": 54, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": true, - "hideZero": true, + "hideEmpty": false, "max": true, "min": false, "rightSide": true, @@ -3938,18 +4366,27 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_kvclient_pull_event_count{changefeed=~\"$changefeed\"}[1m])) by (instance, type)", + "expr": "histogram_quantile(0.99, sum(rate(ticdc_puller_entry_sorter_merge_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,table))", "format": "time_series", + "hide": false, "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{table}}-p99", "refId": "A" + }, + { + "expr": "histogram_quantile(0.90, sum(rate(ticdc_puller_entry_sorter_merge_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,table))", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "{{table}}-p90", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "EventFeed Receive Event Count By Instance", + "title": "entry sorter merge duration", "tooltip": { "shared": true, "sort": 0, @@ -3965,9 +4402,9 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, - "logBase": 1, + "logBase": 2, "max": null, "min": null, "show": true @@ -3992,22 +4429,22 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "Puller entry sorter output channel size", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 16 + "y": 66 }, "hiddenSeries": false, - "id": 31, + "id": 41, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": true, - "hideZero": true, + "hideEmpty": false, "max": true, "min": false, "rightSide": true, @@ -4033,10 +4470,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_kvclient_send_event_count{changefeed=~\"$changefeed\"}[1m])) by (instance, type)", + "expr": "sum(ticdc_puller_entry_sorter_output_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{capture}}-{{table}}", "refId": "A" } ], @@ -4044,7 +4481,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Dispatch Event Count By Instance", + "title": "entry sorter output chan size", "tooltip": { "shared": true, "sort": 0, @@ -4087,22 +4524,23 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Puller event channel size", + "description": "request count of etcd operation", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 16 + "y": 66 }, "hiddenSeries": false, - "id": 40, + "id": 102, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, + "hideEmpty": true, + "hideZero": true, "max": true, "min": false, "rightSide": true, @@ -4128,10 +4566,11 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_puller_event_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", + "expr": "sum(rate(ticdc_etcd_request_count{capture=~\"$capture\"}[1m])) by (capture, type)", "format": "time_series", + "interval": "", "intervalFactor": 1, - "legendFormat": "{{capture}}", + "legendFormat": "{{capture}}-{{type}}", "refId": "A" } ], @@ -4139,7 +4578,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "puller event chan size", + "title": "Etcd Request Count By Instance", "tooltip": { "shared": true, "sort": 0, @@ -4175,33 +4614,47 @@ "align": false, "alignLevel": null } - }, + } + ], + "title": "Events", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 3 + }, + "id": 58, + "panels": [ { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Puller entry channel buffer size", + "decimals": 1, "fill": 1, - "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 0, - "y": 23 + "y": 16 }, - "hiddenSeries": false, - "id": 43, + "id": 60, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, "max": true, "min": false, "rightSide": true, "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -4209,12 +4662,9 @@ "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, "paceLength": 10, "percentage": false, - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -4223,18 +4673,19 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_puller_output_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", + "expr": "sum(rate(tikv_thread_cpu_seconds_total{instance=~\"$tikv_instance\", name=~\"cdc\"}[1m])) by (instance)", "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}} - {{table}}", - "refId": "A" + "intervalFactor": 2, + "legendFormat": "{{instance}}", + "refId": "A", + "step": 4 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "puller output chan size", + "title": "CDC endpoint CPU", "tooltip": { "shared": true, "sort": 0, @@ -4250,7 +4701,7 @@ }, "yaxes": [ { - "format": "short", + "format": "percentunit", "label": null, "logBase": 1, "max": null, @@ -4277,26 +4728,26 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Puller in memory buffer size", + "decimals": 1, "fill": 1, - "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 12, - "y": 23 + "y": 16 }, - "hiddenSeries": false, - "id": 50, + "id": 62, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, "max": true, "min": false, "rightSide": true, "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -4304,32 +4755,43 @@ "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, "paceLength": 10, "percentage": false, - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/.*tso/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_puller_mem_buffer_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", + "expr": "sum(rate(tikv_thread_cpu_seconds_total{instance=~\"$tikv_instance\", name=~\"cdcwkr.*\"}[1m])) by (instance)", "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}} - {{table}}", - "refId": "A" + "intervalFactor": 2, + "legendFormat": "{{instance}} - worker", + "refId": "A", + "step": 4 + }, + { + "expr": "sum(rate(tikv_thread_cpu_seconds_total{instance=~\"$tikv_instance\", name=~\"tso\"}[1m])) by (instance)", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{instance}} - tso", + "refId": "B", + "step": 4 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "puller mem buffer size", + "title": "CDC worker CPU", "tooltip": { "shared": true, "sort": 0, @@ -4345,7 +4807,7 @@ }, "yaxes": [ { - "format": "short", + "format": "percentunit", "label": null, "logBase": 1, "max": null, @@ -4353,7 +4815,7 @@ "show": true }, { - "format": "short", + "format": "percentunit", "label": null, "logBase": 1, "max": null, @@ -4367,99 +4829,141 @@ } }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, + "columns": [ + { + "text": "Current", + "value": "current" + }, + { + "text": "Max", + "value": "max" + } + ], "datasource": "${DS_TEST-CLUSTER}", - "description": "forwarded resolved ts in a single puller", - "fill": 1, - "fillGradient": 0, + "fontSize": "90%", "gridPos": { - "h": 7, - "w": 12, + "h": 8, + "w": 7, "x": 0, - "y": 30 + "y": 24 }, - "hiddenSeries": false, - "id": 52, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": false, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, + "hideTimeOverride": true, + "id": 64, "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": 2, + "desc": false }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, + "styles": [ + { + "alias": "Instance", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "decimals": 2, + "mappingType": 1, + "pattern": "Metric", + "thresholds": [], + "type": "string", + "unit": "short", + "valueMaps": [] + }, + { + "alias": "Time", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "MM-DD HH:mm:ss.SSS", + "decimals": 2, + "mappingType": 1, + "pattern": "Max", + "thresholds": [], + "type": "date", + "unit": "short" + } + ], "targets": [ { - "expr": "sum(ticdc_puller_resolved_ts{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", + "expr": "tikv_cdc_min_resolved_ts{instance=~\"$tikv_instance\"}", "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-{{table}}", + "intervalFactor": 2, + "legendFormat": "{{instance}}", "refId": "A" + }, + { + "expr": "max(pd_cluster_tso) * 1000", + "format": "time_series", + "hide": true, + "intervalFactor": 2, + "legendFormat": "current time (s)", + "refId": "B" } ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "puller resolved ts", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" + "timeFrom": "1s", + "title": "Min resolved ts", + "transform": "timeseries_aggregations", + "type": "table" + }, + { + "columns": [ + { + "text": "Current", + "value": "current" + } + ], + "datasource": "${DS_TEST-CLUSTER}", + "fontSize": "90%", + "gridPos": { + "h": 8, + "w": 5, + "x": 7, + "y": 24 }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] + "hideTimeOverride": true, + "id": 66, + "links": [], + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": 0, + "desc": false }, - "yaxes": [ + "styles": [ { - "format": "dateTimeAsIso", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, + "alias": "Option", + "colorMode": null, + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "mappingType": 2, + "pattern": "Metric", + "preserveFormat": false, + "sanitize": false, + "type": "number" + } + ], + "targets": [ { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "expr": "tikv_cdc_min_resolved_ts_region{instance=~\"$tikv_instance\"}", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{instance}}", + "refId": "A" } ], - "yaxis": { - "align": false, - "alignLevel": null - } + "timeFrom": "1s", + "title": "Min resolved Region", + "transform": "timeseries_aggregations", + "type": "table" }, { "aliasColors": {}, @@ -4467,26 +4971,29 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Puller entry sorter unsoreted items size", + "decimals": 1, + "description": "", "fill": 1, - "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 12, - "y": 30 + "y": 24 }, - "hiddenSeries": false, - "id": 51, + "id": 70, "legend": { "alignAsTable": true, "avg": false, "current": true, "hideEmpty": false, + "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, + "sideWidth": null, + "sort": null, + "sortDesc": null, "total": false, "values": true }, @@ -4494,12 +5001,9 @@ "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, "paceLength": 10, "percentage": false, - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -4508,10 +5012,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_puller_entry_sorter_unsorted_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", + "expr": "histogram_quantile(0.99999, sum(rate(tikv_cdc_resolved_ts_gap_seconds_bucket{instance=~\"$tikv_instance\"}[1m])) by (le, instance))", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}-{{table}}", + "legendFormat": "{{instance}}", "refId": "A" } ], @@ -4519,7 +5023,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "entry sorter unsorted size", + "title": "99.9% Resolved ts lag duration", "tooltip": { "shared": true, "sort": 0, @@ -4535,9 +5039,9 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, - "logBase": 1, + "logBase": 10, "max": null, "min": null, "show": true @@ -4557,99 +5061,83 @@ } }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", "datasource": "${DS_TEST-CLUSTER}", - "description": "Puller entry sorter resolved channel size", - "fill": 1, - "fillGradient": 0, + "description": "The time consumed to CDC incremental scan", "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 0, - "y": 37 + "y": 32 }, - "hiddenSeries": false, - "id": 42, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 68, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, "max": true, "min": false, "rightSide": true, "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "maxPerRow": 3, + "repeat": null, + "repeatDirection": "h", + "reverseYBuckets": false, "targets": [ { - "expr": "sum(ticdc_puller_entry_sorter_resolved_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-{{table}}", + "expr": "sum(rate(tikv_cdc_scan_duration_seconds_bucket{instance=~\"$tikv_instance\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", "refId": "A" } ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "entry sorter resolved chan size", + "title": "Initial scan duration", "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" + "show": true, + "showHistogram": true }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, "show": true, - "values": [] + "splitFactor": null }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null }, { "aliasColors": {}, @@ -4657,26 +5145,29 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Puller entry sorter output channel size", + "decimals": 1, + "description": "", "fill": 1, - "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 12, - "y": 37 + "y": 32 }, - "hiddenSeries": false, - "id": 41, + "id": 72, "legend": { "alignAsTable": true, "avg": false, "current": true, "hideEmpty": false, + "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -4684,12 +5175,9 @@ "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, "paceLength": 10, "percentage": false, - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -4698,10 +5186,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_puller_entry_sorter_output_chan_size{changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture, table)", + "expr": "histogram_quantile(0.999, sum(rate(tikv_cdc_scan_duration_seconds_bucket{instance=~\"$tikv_instance\"}[1m])) by (le, instance))", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}-{{table}}", + "legendFormat": "{{instance}}", "refId": "A" } ], @@ -4709,7 +5197,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "entry sorter output chan size", + "title": "99.9% Initial scan duration", "tooltip": { "shared": true, "sort": 0, @@ -4725,7 +5213,7 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -4738,7 +5226,7 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -4752,26 +5240,30 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Bucketed histogram of processing time (s) of merge in entry sorter", - "fill": 1, - "fillGradient": 0, + "decimals": 1, + "description": "The memory usage per TiKV instance", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 0, - "y": 44 + "y": 40 }, - "hiddenSeries": false, - "id": 53, + "id": 74, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, "max": true, "min": false, "rightSide": true, "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, "total": false, "values": true }, @@ -4779,12 +5271,9 @@ "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, "paceLength": 10, "percentage": false, - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -4793,27 +5282,40 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_puller_entry_sorter_sort_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,table))", + "expr": "avg(process_resident_memory_bytes{instance=~\"$tikv_instance\", job=\"tikv\"}) by (instance)", "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{table}}-p99", - "refId": "A" + "hide": true, + "intervalFactor": 2, + "legendFormat": "tikv-{{instance}}", + "refId": "A", + "step": 10 }, { - "expr": "histogram_quantile(0.90, sum(rate(ticdc_puller_entry_sorter_sort_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,table))", + "expr": "avg(process_resident_memory_bytes{instance=~\"$tikv_instance\", job=\"cdc\"}) by (instance)", "format": "time_series", "hide": true, - "intervalFactor": 1, - "legendFormat": "{{table}}-p90", - "refId": "B" + "intervalFactor": 2, + "legendFormat": "cdc-{{instance}}", + "refId": "B", + "step": 10 + }, + { + "expr": "(avg(process_resident_memory_bytes{instance=~\"$tikv_instance\", job=\"tikv\"}) by (instance)) - (avg(tikv_engine_block_cache_size_bytes{instance=~\"$tikv_instance\", db=\"kv\"}) by(instance))", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "tikv-{{instance}}", + "refId": "C", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "entry sorter sort duration", + "title": "Memory without block cache", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -4828,11 +5330,11 @@ }, "yaxes": [ { - "format": "short", + "format": "bytes", "label": null, - "logBase": 2, + "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -4855,26 +5357,30 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Bucketed histogram of processing time (s) of merge in entry sorter.", - "fill": 1, - "fillGradient": 0, + "decimals": 1, + "description": "The memory usage per TiKV instance", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 12, - "y": 44 + "y": 40 }, - "hiddenSeries": false, - "id": 54, + "id": 76, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, "max": true, "min": false, "rightSide": true, "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -4882,12 +5388,9 @@ "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, "paceLength": 10, "percentage": false, - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -4896,28 +5399,22 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_puller_entry_sorter_merge_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,table))", + "expr": "avg(tikv_cdc_min_resolved_ts{instance=~\"$tikv_instance\", job=\"tikv\"}) by (instance)", "format": "time_series", "hide": false, - "intervalFactor": 1, - "legendFormat": "{{table}}-p99", - "refId": "A" - }, - { - "expr": "histogram_quantile(0.90, sum(rate(ticdc_puller_entry_sorter_merge_bucket{changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,table))", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "{{table}}-p90", - "refId": "B" + "intervalFactor": 2, + "legendFormat": "tikv-{{instance}}", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "entry sorter merge duration", + "title": "CDC pending bytes in memory", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -4932,9 +5429,9 @@ }, "yaxes": [ { - "format": "short", + "format": "none", "label": null, - "logBase": 2, + "logBase": 1, "max": null, "min": null, "show": true @@ -4959,27 +5456,30 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "request count of etcd operation", - "fill": 1, - "fillGradient": 0, + "decimals": 1, + "description": "The memory usage per TiKV instance", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 0, - "y": 121 + "y": 48 }, - "hiddenSeries": false, - "id": 97, + "id": 78, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": true, - "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, + "sideWidth": null, + "sort": null, + "sortDesc": null, "total": false, "values": true }, @@ -4987,12 +5487,9 @@ "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, "paceLength": 10, "percentage": false, - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -5001,20 +5498,22 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_etcd_request_count{capture=~\"$capture\"}[1m])) by (instance, type)", + "expr": "avg(tikv_cdc_captured_region_total{instance=~\"$tikv_instance\"}) by (instance)", "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", - "refId": "A" + "hide": false, + "intervalFactor": 2, + "legendFormat": "tikv-{{instance}}", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Etcd Request Count By Instance", + "title": "Captured region count", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -5029,11 +5528,11 @@ }, "yaxes": [ { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -5051,7 +5550,7 @@ } } ], - "title": "Events", + "title": "TiKV", "type": "row" } ], @@ -5067,13 +5566,13 @@ "datasource": "${DS_TEST-CLUSTER}", "definition": "label_values(ticdc_processor_resolved_ts, changefeed)", "hide": 0, - "includeAll": true, + "includeAll": false, "label": "Changefeed", "multi": false, "name": "changefeed", "options": [], "query": "label_values(ticdc_processor_resolved_ts, changefeed)", - "refresh": 1, + "refresh": 2, "regex": "", "skipUrlSync": false, "sort": 0, @@ -5095,7 +5594,7 @@ "name": "capture", "options": [], "query": "label_values(ticdc_processor_resolved_ts, capture)", - "refresh": 1, + "refresh": 2, "regex": "", "skipUrlSync": false, "sort": 0, @@ -5117,7 +5616,7 @@ "name": "tikv_instance", "options": [], "query": "label_values(tikv_engine_size_bytes, instance)", - "refresh": 1, + "refresh": 2, "regex": "", "skipUrlSync": false, "sort": 1, @@ -5126,6 +5625,60 @@ "tagsQuery": "", "type": "query", "useTags": false + }, + { + "allValue": "9999999999", + "current": { + "selected": false, + "tags": [], + "text": "All", + "value": "$__all" + }, + "hide": 0, + "includeAll": true, + "label": "Latency spike (s) >", + "multi": false, + "name": "spike_threshold", + "options": [ + { + "selected": true, + "text": "All", + "value": "$__all" + }, + { + "selected": false, + "text": "1", + "value": "1" + }, + { + "selected": false, + "text": "3", + "value": "3" + }, + { + "selected": false, + "text": "5", + "value": "5" + }, + { + "selected": false, + "text": "10", + "value": "10" + }, + { + "selected": false, + "text": "60", + "value": "60" + }, + { + "selected": false, + "text": "300", + "value": "300" + } + ], + "query": "1, 3, 5, 10, 60, 300", + "skipUrlSync": false, + "type": "custom" } ] }, @@ -5161,5 +5714,5 @@ "timezone": "browser", "title": "Test-Cluster-CDC", "uid": "YiGL8hBZ1", - "version": 7 -} + "version": 9 +} \ No newline at end of file diff --git a/pkg/util/check.go b/pkg/version/check.go similarity index 99% rename from pkg/util/check.go rename to pkg/version/check.go index 5b3a0a929ea..51f5b42257d 100644 --- a/pkg/util/check.go +++ b/pkg/version/check.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package util +package version import ( "context" diff --git a/pkg/util/check_test.go b/pkg/version/check_test.go similarity index 91% rename from pkg/util/check_test.go rename to pkg/version/check_test.go index 3763e28954c..580f867f1f0 100644 --- a/pkg/util/check_test.go +++ b/pkg/version/check_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package util +package version import ( "context" @@ -129,3 +129,17 @@ func (s *checkSuite) TestCompareVersion(c *check.C) { c.Assert(semver.New(removeVAndHash("v2.1.0-rc.1-7-g38c939f-dirty")). Compare(*semver.New("2.1.0-rc.1")), check.Equals, 0) } + +func (s *checkSuite) TestReleaseSemver(c *check.C) { + cases := []struct{ releaseVersion, releaseSemver string }{ + {"None", ""}, + {"HEAD", ""}, + {"v4.0.5", "4.0.5"}, + {"v4.0.2-152-g62d7075-dev", "4.0.2"}, + } + + for _, cs := range cases { + ReleaseVersion = cs.releaseVersion + c.Assert(ReleaseSemver(), check.Equals, cs.releaseSemver, check.Commentf("%v", cs)) + } +} diff --git a/pkg/util/version.go b/pkg/version/version.go similarity index 81% rename from pkg/util/version.go rename to pkg/version/version.go index 5874469951a..c713053ee5d 100644 --- a/pkg/util/version.go +++ b/pkg/version/version.go @@ -11,11 +11,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -package util +package version import ( "fmt" + "github.com/coreos/go-semver/semver" "github.com/pingcap/log" "go.uber.org/zap" ) @@ -29,6 +30,17 @@ var ( GoVersion = "None" ) +// ReleaseSemver returns a valid Semantic Versions or an empty if the +// ReleaseVersion is not set at compile time. +func ReleaseSemver() string { + s := removeVAndHash(ReleaseVersion) + v, err := semver.NewVersion(s) + if err != nil { + return "" + } + return v.String() +} + // LogVersionInfo prints the CDC version information. func LogVersionInfo() { log.Info("Welcome to Change Data Capture (CDC)", diff --git a/tests/new_ci_collation/conf/changefeed.toml b/tests/new_ci_collation/conf/changefeed.toml new file mode 100644 index 00000000000..4293a79e451 --- /dev/null +++ b/tests/new_ci_collation/conf/changefeed.toml @@ -0,0 +1 @@ +enable-old-value = true diff --git a/tests/new_ci_collation/conf/diff_config.toml b/tests/new_ci_collation/conf/diff_config.toml new file mode 100644 index 00000000000..bd6162d0813 --- /dev/null +++ b/tests/new_ci_collation/conf/diff_config.toml @@ -0,0 +1,27 @@ +# diff Configuration. + +log-level = "info" +chunk-size = 10 +check-thread-count = 4 +sample-percent = 100 +use-rowid = false +use-checksum = true +fix-sql-file = "fix.sql" + +# tables need to check. +[[check-tables]] + schema = "new_ci_collation_test" + tables = ["~.*"] + +[[source-db]] + host = "127.0.0.1" + port = 4000 + user = "root" + password = "" + instance-id = "source-1" + +[target-db] + host = "127.0.0.1" + port = 3306 + user = "root" + password = "" diff --git a/tests/new_ci_collation/conf/tidb_config.toml b/tests/new_ci_collation/conf/tidb_config.toml new file mode 100644 index 00000000000..bc23af7783a --- /dev/null +++ b/tests/new_ci_collation/conf/tidb_config.toml @@ -0,0 +1,2 @@ +alter-primary-key = true +new_collations_enabled_on_first_bootstrap = true diff --git a/tests/new_ci_collation/data/test1.sql b/tests/new_ci_collation/data/test1.sql new file mode 100644 index 00000000000..a689dbae138 --- /dev/null +++ b/tests/new_ci_collation/data/test1.sql @@ -0,0 +1,49 @@ +drop database if exists `new_ci_collation_test`; +create database `new_ci_collation_test`; +use `new_ci_collation_test`; + +CREATE TABLE t1 ( + a varchar(20) charset utf8mb4 collate utf8mb4_general_ci primary key, + b int default 10 +); + +CREATE TABLE t2 ( + a varchar(10) charset utf8 collate utf8_general_ci, primary key(a), + b int default 10 +); + +CREATE TABLE t3 ( + id int primary key auto_increment, + a varchar(20) charset utf8mb4 collate utf8mb4_general_ci, + b int default 10 +); + +CREATE TABLE t4 ( + a int primary key, + b varchar(10) charset utf8mb4 collate utf8mb4_general_ci, + c varchar(10) charset utf8 collate utf8_general_ci, + d varchar(10) not null, + unique key d(d) +); + +CREATE TABLE t5 ( + a varchar(10) charset utf8mb4 collate utf8mb4_bin, primary key(a), + b int default 10 +); + + +insert into t1 (a) values ('A'),(' A'),('A\t'),('b'),('bA'),('bac'),('ab'); +insert into t1 (a) values ('😉'); +insert into t2 (a) values ('A'),(' A'),('A\t'),('b'),('bA'),('bac'),('ab'); +insert into t3 (a) values ('A'),('A '),('A '),(' A'),('A\t'),('A\t '); +insert into t3 (a) values ('a'),('a '),('a '),(' a'),('a\t'),('a\t '); +insert into t3 (a) values ('B'),('B '),('B '),(' B'),('B\t'),('B\t '); +insert into t3 (a) values ('b'),('b '),('b '),(' b'),('b\t'),('b\t '); +insert into t4 values (1,'A','A','1'),(2,'a\t','a\t','2'),(3,'ab','ab','3'),(4,'abc','abc','4'); +insert into t5 (a) values ('😉'); +insert into t5 (a) values ('a'),('A'),(' a'),(' A'),('a\t'),('ab'),('Ab'); +update t1 set b = b + 1; +update t2 set b = 13; +update t3 set b = 11 where a > 'A'; +drop index `primary` on t4; +update t5 set b = 12; diff --git a/tests/new_ci_collation/data/test2.sql b/tests/new_ci_collation/data/test2.sql new file mode 100644 index 00000000000..40a1a6b0331 --- /dev/null +++ b/tests/new_ci_collation/data/test2.sql @@ -0,0 +1,6 @@ +use `new_ci_collation_test`; +delete from t1; +delete from t2; +delete from t3; +delete from t4; +delete from t5; diff --git a/tests/new_ci_collation/run.sh b/tests/new_ci_collation/run.sh new file mode 100755 index 00000000000..fc7d26e899a --- /dev/null +++ b/tests/new_ci_collation/run.sh @@ -0,0 +1,47 @@ +#!/bin/bash + +set -e + +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() { + rm -rf $WORK_DIR && mkdir -p $WORK_DIR + + start_tidb_cluster --workdir $WORK_DIR --tidb-config $CUR/conf/tidb_config.toml + + cd $WORK_DIR + + # record tso before we create tables to skip the system table DDLs + start_ts=$(cdc cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) + + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + + TOPIC_NAME="ticdc-new_ci_collation-test-$RANDOM" + case $SINK_TYPE in + kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4";; + *) SINK_URI="mysql://root@127.0.0.1:3306/?safe-mode=true";; + esac + cdc cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --config $CUR/conf/changefeed.toml + if [ "$SINK_TYPE" == "kafka" ]; then + run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4" + fi + + run_sql_file $CUR/data/test1.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} + for i in $(seq 1 5); do + table="new_ci_collation_test.t$i" + check_table_exists $table ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + done + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml + run_sql_file $CUR/data/test2.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml + + cleanup_process $CDC_BINARY +} + +trap stop_tidb_cluster EXIT +run $* +echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" diff --git a/tests/run.sh b/tests/run.sh index 3f4cb240e57..9e3928c12c4 100755 --- a/tests/run.sh +++ b/tests/run.sh @@ -41,12 +41,15 @@ run_case() { bash "$script" "$sink_type" } -test_case=$1 -sink_type=$2 +sink_type=$1 + +set +eu +test_case=$2 if [ -z "$test_case" ]; then test_case="*" fi +set -eu if [ "$test_case" == "*" ]; then for script in $CUR/*/run.sh; do