diff --git a/.golangci.yml b/.golangci.yml index c018f059d0..00decdaa79 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -11,6 +11,8 @@ linters: - bodyclose # copyloopvar: detects redundant "x := x" in Go 1.22+ loop variables. - copyloopvar + # depguard: forbids direct imports that bypass repository wrappers. + - depguard # durationcheck: checks for suspicious duration multiplication. - durationcheck # errname: checks sentinel error naming (Err prefix, Error suffix). @@ -19,6 +21,8 @@ linters: - errorlint # gosec: inspects source code for security problems. - gosec + # importas: enforces consistent import aliases. + - importas # misspell: finds commonly misspelled English words. - misspell # nilerr: finds code that returns nil even if err is not nil. @@ -35,6 +39,23 @@ linters: - unconvert settings: + depguard: + rules: + no-direct-errors: + files: + - $all + - '!pkg/errors/**' + deny: + - pkg: errors$ + desc: use github.com/pingcap/ticdc/pkg/errors outside pkg/errors + - pkg: github.com/pingcap/errors$ + desc: use github.com/pingcap/ticdc/pkg/errors outside pkg/errors + + importas: + alias: + - pkg: github.com/pingcap/ticdc/pkg/errors + alias: "" + revive: rules: - name: blank-imports @@ -67,10 +88,12 @@ linters: - G104 # ST1000: don't require package comments. + # ST1003: don't enforce naming conventions on legacy identifiers. staticcheck: checks: - all - -ST1000 + - -ST1003 exclusions: # Strict: exclude all known generated file patterns. diff --git a/Makefile b/Makefile index bcf33d09ef..ab89082295 100644 --- a/Makefile +++ b/Makefile @@ -1,6 +1,6 @@ # Phony targets are targets that are not associated with files. # Add new phony targets here to make them available in the `make` command. -.PHONY: clean fmt check check-static tidy \ +.PHONY: clean fmt check check-static local-static-check tidy \ generate-protobuf generate_mock \ cdc kafka_consumer storage_consumer pulsar_consumer filter_helper \ prepare_test_binaries \ @@ -320,6 +320,14 @@ else tools/bin/golangci-lint run --timeout 10m0s endif +# Lint only code changed on the current branch (vs upstream/master by default). +# Override base with LINT_BASE=. +# make local-static-check +# make local-static-check LINT_BASE=HEAD~3 +local-static-check: tools/bin/golangci-lint + $(eval BASE := $(if $(LINT_BASE),$(LINT_BASE),upstream/master)) + tools/bin/golangci-lint run --timeout 10m0s --new-from-rev=$(BASE) + check-ticdc-dashboard: @echo "check-ticdc-dashboard" @./scripts/check-ticdc-dashboard.sh diff --git a/api/status/status.go b/api/status/status.go index c3e08441fa..b0d05cd7f2 100644 --- a/api/status/status.go +++ b/api/status/status.go @@ -87,7 +87,7 @@ func (h *statusAPI) handleDebugInfo(w http.ResponseWriter, req *http.Request) { h.writeEtcdInfo(ctx, h.server.GetEtcdClient(), w) } -func (h *statusAPI) handleStatus(w http.ResponseWriter, req *http.Request) { +func (h *statusAPI) handleStatus(w http.ResponseWriter, _ *http.Request) { st := status{ Version: version.ReleaseVersion, GitHash: version.GitHash, diff --git a/api/v2/model.go b/api/v2/model.go index 23324d9221..1aee8e00c8 100644 --- a/api/v2/model.go +++ b/api/v2/model.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/integrity" "github.com/pingcap/ticdc/pkg/liveness" - "github.com/pingcap/ticdc/pkg/security" "github.com/pingcap/ticdc/pkg/util" ) @@ -85,12 +84,6 @@ type VerifyTableConfig struct { SinkURI string `json:"sink_uri"` } -func getDefaultVerifyTableConfig() *VerifyTableConfig { - return &VerifyTableConfig{ - ReplicaConfig: GetDefaultReplicaConfig(), - } -} - // ResumeChangefeedConfig is used by resume changefeed api type ResumeChangefeedConfig struct { PDConfig @@ -1350,18 +1343,6 @@ type SyncedStatus struct { Info string `json:"info"` } -// toCredential generates a security.Credential from a PDConfig -func (cfg *PDConfig) toCredential() *security.Credential { - credential := &security.Credential{ - CAPath: cfg.CAPath, - CertPath: cfg.CertPath, - KeyPath: cfg.KeyPath, - } - credential.CertAllowedCN = make([]string, len(cfg.CertAllowedCN)) - copy(credential.CertAllowedCN, cfg.CertAllowedCN) - return credential -} - // Marshal returns the json marshal format of a ChangeFeedInfo func (info *ChangeFeedInfo) Marshal() (string, error) { data, err := json.Marshal(info) diff --git a/cmd/cdc/cli/cli_changefeed_helper.go b/cmd/cdc/cli/cli_changefeed_helper.go index 5a655672e9..83911c970d 100644 --- a/cmd/cdc/cli/cli_changefeed_helper.go +++ b/cmd/cdc/cli/cli_changefeed_helper.go @@ -14,14 +14,12 @@ package cli import ( - "bufio" "fmt" - "os" "strings" "time" v2 "github.com/pingcap/ticdc/api/v2" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/errors" "github.com/spf13/cobra" "github.com/tikv/client-go/v2/oracle" ) @@ -32,15 +30,6 @@ const ( tsGapWarning = 86400 * 1000 ) -func readInput() (string, error) { - reader := bufio.NewReader(os.Stdin) - msg, err := reader.ReadString('\n') - if err != nil { - return "", err - } - return strings.TrimSpace(msg), nil -} - func readYOrN(cmd *cobra.Command) bool { var yOrN string _, err := fmt.Scan(&yOrN) @@ -66,7 +55,7 @@ func confirmLargeDataGap(cmd *cobra.Command, currentPhysical int64, startTs uint confirmed := readYOrN(cmd) if !confirmed { cmd.Printf("Abort changefeed %s.\n", command) - return cerror.ErrCliAborted.FastGenByArgs(fmt.Sprintf("cli changefeed %s", command)) + return errors.ErrCliAborted.FastGen("cli changefeed %s", command) } } @@ -84,7 +73,7 @@ func confirmOverwriteCheckpointTs( confirmed := readYOrN(cmd) if !confirmed { cmd.Printf("Abort changefeed resume.\n") - return cerror.ErrCliAborted.FastGenByArgs("cli changefeed resume") + return errors.ErrCliAborted.FastGenByArgs("cli changefeed resume") } return nil @@ -99,7 +88,7 @@ func confirmIgnoreIneligibleTables(cmd *cobra.Command) (bool, error) { confirmed := readYOrN(cmd) if !confirmed { cmd.Printf("No changefeed is created because you don't want to ignore some tables.\n") - return false, cerror.ErrCliAborted.FastGenByArgs("cli changefeed create") + return false, errors.ErrCliAborted.FastGenByArgs("cli changefeed create") } return true, nil diff --git a/cmd/config-converter/main.go b/cmd/config-converter/main.go index 0eb604dbbf..7546cdeb68 100644 --- a/cmd/config-converter/main.go +++ b/cmd/config-converter/main.go @@ -58,7 +58,7 @@ func main() { } } -func run(cmd *cobra.Command, args []string) { +func run(_ *cobra.Command, _ []string) { if cfgPath != "" && modelPath != "" { fmt.Fprintln(os.Stderr, "can't specify both config and model") os.Exit(ExitCodeInvalidFlag) diff --git a/cmd/storage-consumer/consumer.go b/cmd/storage-consumer/consumer.go index 2de4d3c0c4..623e41b270 100644 --- a/cmd/storage-consumer/consumer.go +++ b/cmd/storage-consumer/consumer.go @@ -21,7 +21,6 @@ import ( "strings" "time" - "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/ticdc/cmd/util" "github.com/pingcap/ticdc/downstreamadapter/sink" @@ -29,6 +28,7 @@ import ( commonType "github.com/pingcap/ticdc/pkg/common" "github.com/pingcap/ticdc/pkg/common/event" "github.com/pingcap/ticdc/pkg/config" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/sink/cloudstorage" "github.com/pingcap/ticdc/pkg/sink/codec/canal" "github.com/pingcap/ticdc/pkg/sink/codec/common" @@ -244,7 +244,7 @@ func (c *consumer) getNewFiles( origDMLIdxMap[k] = m } - err := c.externalStorage.WalkDir(ctx, opt, func(path string, size int64) error { + err := c.externalStorage.WalkDir(ctx, opt, func(path string, _ int64) error { if cloudstorage.IsSchemaFile(path) { err := c.parseSchemaFilePath(ctx, path) if err != nil { @@ -713,7 +713,9 @@ func (c *consumer) handleNewFiles( } } } - c.flushDMLEvents(ctx, tableID) + if err := c.flushDMLEvents(ctx, tableID); err != nil { + return err + } } return nil diff --git a/cmd/storage-consumer/main.go b/cmd/storage-consumer/main.go index c8aa270301..70256977f4 100644 --- a/cmd/storage-consumer/main.go +++ b/cmd/storage-consumer/main.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/ticdc/pkg/config" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/logger" "github.com/pingcap/ticdc/pkg/version" "go.uber.org/zap" @@ -103,7 +104,7 @@ func main() { if consumer != nil { consumer.sink.Close() } - if err != nil && err != context.Canceled { + if err != nil && !errors.Is(err, context.Canceled) { return 1 } return 0 diff --git a/coordinator/controller.go b/coordinator/controller.go index d4214d53e1..73e579df0e 100644 --- a/coordinator/controller.go +++ b/coordinator/controller.go @@ -174,7 +174,7 @@ func NewController( // detect the capture changes c.nodeManager.RegisterNodeChangeHandler( nodeChangeHandlerID, - func(allNodes map[node.ID]*node.Info) { + func(_ map[node.ID]*node.Info) { c.nodeChanged.Lock() defer c.nodeChanged.Unlock() c.nodeChanged.changed = true @@ -355,9 +355,14 @@ func (c *Controller) RequestResolvedTsFromLogCoordinator(ctx context.Context, ch changefeedID := c.changefeedDB.GetChangefeedIDByName(changefeedDisplayName) ids := c.nodeManager.GetAliveNodeIDs() for _, id := range ids { - c.messageCenter.SendEvent(messaging.NewSingleTargetMessage(id, messaging.LogCoordinatorTopic, &heartbeatpb.LogCoordinatorResolvedTsRequest{ + if err := c.messageCenter.SendEvent(messaging.NewSingleTargetMessage(id, messaging.LogCoordinatorTopic, &heartbeatpb.LogCoordinatorResolvedTsRequest{ ChangefeedID: changefeedID.ToPB(), - })) + })); err != nil { + log.Warn("failed to request resolved ts from log coordinator", + zap.Stringer("target", id), + zap.String("changefeed", changefeedID.DisplayName.String()), + zap.Error(err)) + } } // wait for some time to get the resolved ts @@ -686,7 +691,7 @@ func (c *Controller) CreateChangefeed(ctx context.Context, info *config.ChangeFe return errors.Trace(ctx.Err()) case <-ticker.C: log.Warn("changefeed is in scheduling, wait a moment", zap.String("changefeed", info.ChangefeedID.DisplayName.String())) - count += 1 + count++ } } @@ -728,7 +733,7 @@ func (c *Controller) RemoveChangefeed(ctx context.Context, id common.ChangeFeedI case <-ctx.Done(): return 0, errors.Trace(ctx.Err()) case <-ticker.C: - count += 1 + count++ log.Info("wait for stop changefeed operator finished", zap.Int("count", count), zap.Any("id", id)) } } @@ -766,7 +771,7 @@ func (c *Controller) PauseChangefeed(ctx context.Context, id common.ChangeFeedID case <-ctx.Done(): return errors.Trace(ctx.Err()) case <-ticker.C: - count += 1 + count++ log.Info("wait for stop changefeed operator finished", zap.Int("count", count), zap.Any("id", id)) } } diff --git a/coordinator/coordinator_test.go b/coordinator/coordinator_test.go index d8f08c028b..608a1bf348 100644 --- a/coordinator/coordinator_test.go +++ b/coordinator/coordinator_test.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/ticdc/pkg/common" appcontext "github.com/pingcap/ticdc/pkg/common/context" "github.com/pingcap/ticdc/pkg/config" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/etcd" "github.com/pingcap/ticdc/pkg/eventservice" "github.com/pingcap/ticdc/pkg/messaging" @@ -682,7 +683,7 @@ func TestConcurrentStopAndSendEvents(t *testing.T) { ctxRun, cancelRun := context.WithCancel(ctx) go func() { err := cr.Run(ctxRun) - if err != nil && err != context.Canceled { + if err != nil && !errors.Is(err, context.Canceled) { t.Errorf("Coordinator Run returned unexpected error: %v", err) } }() @@ -715,7 +716,7 @@ func TestConcurrentStopAndSendEvents(t *testing.T) { // Use recvMessages to send event to channel err := co.recvMessages(ctx, msg) - if err != nil && err != context.Canceled { + if err != nil && !errors.Is(err, context.Canceled) { t.Logf("Failed to send event in goroutine %d: %v", id, err) } diff --git a/downstreamadapter/dispatchermanager/dispatcher_manager_redo.go b/downstreamadapter/dispatchermanager/dispatcher_manager_redo.go index 94ba6ffac5..9f6873af13 100644 --- a/downstreamadapter/dispatchermanager/dispatcher_manager_redo.go +++ b/downstreamadapter/dispatchermanager/dispatcher_manager_redo.go @@ -139,7 +139,7 @@ func (e *DispatcherManager) getRedoEventCollectorBatchCountAndBytes(redoSink *re return batchCount, batchBytes } -func (e *DispatcherManager) newRedoDispatchers(infos map[common.DispatcherID]dispatcherCreateInfo, removeDDLTs bool) error { +func (e *DispatcherManager) newRedoDispatchers(infos map[common.DispatcherID]dispatcherCreateInfo, _ bool) error { start := time.Now() batchCount, batchBytes := e.getRedoEventCollectorBatchCountAndBytes(e.redoSink) diff --git a/downstreamadapter/dispatchermanager/helper.go b/downstreamadapter/dispatchermanager/helper.go index 35e07a4765..5db7e3480c 100644 --- a/downstreamadapter/dispatchermanager/helper.go +++ b/downstreamadapter/dispatchermanager/helper.go @@ -447,12 +447,12 @@ func createDispatcherByInfo( } } -func (h *SchedulerDispatcherRequestHandler) GetSize(event SchedulerDispatcherRequest) int { return 0 } -func (h *SchedulerDispatcherRequestHandler) IsPaused(event SchedulerDispatcherRequest) bool { +func (h *SchedulerDispatcherRequestHandler) GetSize(_ SchedulerDispatcherRequest) int { return 0 } +func (h *SchedulerDispatcherRequestHandler) IsPaused(_ SchedulerDispatcherRequest) bool { return false } -func (h *SchedulerDispatcherRequestHandler) GetArea(path common.GID, dest *DispatcherManager) int { +func (h *SchedulerDispatcherRequestHandler) GetArea(_ common.GID, _ *DispatcherManager) int { return 0 } @@ -460,7 +460,7 @@ func (h *SchedulerDispatcherRequestHandler) GetMetricLabel(dest *DispatcherManag return dest.changefeedID.String() } -func (h *SchedulerDispatcherRequestHandler) GetTimestamp(event SchedulerDispatcherRequest) dynstream.Timestamp { +func (h *SchedulerDispatcherRequestHandler) GetTimestamp(_ SchedulerDispatcherRequest) dynstream.Timestamp { return 0 } @@ -553,7 +553,7 @@ func (h *HeartBeatResponseHandler) Handle(dispatcherManager *DispatcherManager, func (h *HeartBeatResponseHandler) GetSize(event HeartBeatResponse) int { return 0 } func (h *HeartBeatResponseHandler) IsPaused(event HeartBeatResponse) bool { return false } -func (h *HeartBeatResponseHandler) GetArea(path common.GID, dest *DispatcherManager) int { +func (h *HeartBeatResponseHandler) GetArea(_ common.GID, _ *DispatcherManager) int { return 0 } @@ -609,7 +609,7 @@ func (h *CheckpointTsMessageHandler) Handle(dispatcherManager *DispatcherManager func (h *CheckpointTsMessageHandler) GetSize(event CheckpointTsMessage) int { return 0 } func (h *CheckpointTsMessageHandler) IsPaused(event CheckpointTsMessage) bool { return false } -func (h *CheckpointTsMessageHandler) GetArea(path common.GID, dest *DispatcherManager) int { +func (h *CheckpointTsMessageHandler) GetArea(_ common.GID, _ *DispatcherManager) int { return 0 } @@ -663,7 +663,7 @@ func (h *RedoResolvedTsForwardMessageHandler) Handle(dispatcherManager *Dispatch msg := messages[0] ok := dispatcherManager.SetRedoResolvedTs(msg.ResolvedTs) if ok { - dispatcherManager.dispatcherMap.ForEach(func(id common.DispatcherID, dispatcher *dispatcher.EventDispatcher) { + dispatcherManager.dispatcherMap.ForEach(func(_ common.DispatcherID, dispatcher *dispatcher.EventDispatcher) { dispatcher.HandleCacheEvents() }) } @@ -678,7 +678,7 @@ func (h *RedoResolvedTsForwardMessageHandler) IsPaused(event RedoResolvedTsForwa return false } -func (h *RedoResolvedTsForwardMessageHandler) GetArea(path common.GID, dest *DispatcherManager) int { +func (h *RedoResolvedTsForwardMessageHandler) GetArea(_ common.GID, _ *DispatcherManager) int { return 0 } @@ -738,7 +738,7 @@ func (h *RedoMetaMessageHandler) Handle(dispatcherManager *DispatcherManager, me func (h *RedoMetaMessageHandler) GetSize(event RedoMetaMessage) int { return 0 } func (h *RedoMetaMessageHandler) IsPaused(event RedoMetaMessage) bool { return false } -func (h *RedoMetaMessageHandler) GetArea(path common.GID, dest *DispatcherManager) int { +func (h *RedoMetaMessageHandler) GetArea(_ common.GID, _ *DispatcherManager) int { return 0 } @@ -795,7 +795,7 @@ func (h *MergeDispatcherRequestHandler) Handle(dispatcherManager *DispatcherMana func (h *MergeDispatcherRequestHandler) GetSize(event MergeDispatcherRequest) int { return 0 } func (h *MergeDispatcherRequestHandler) IsPaused(event MergeDispatcherRequest) bool { return false } -func (h *MergeDispatcherRequestHandler) GetArea(path common.GID, dest *DispatcherManager) int { +func (h *MergeDispatcherRequestHandler) GetArea(_ common.GID, _ *DispatcherManager) int { return 0 } diff --git a/downstreamadapter/eventcollector/event_collector.go b/downstreamadapter/eventcollector/event_collector.go index e22ab376b9..6ac31cf645 100644 --- a/downstreamadapter/eventcollector/event_collector.go +++ b/downstreamadapter/eventcollector/event_collector.go @@ -770,14 +770,6 @@ func updateMinUint64MapValue(m map[common.ChangeFeedID]uint64, key common.Change } } -func updateMaxUint64MapValue(m map[common.ChangeFeedID]uint64, key common.ChangeFeedID, value uint64) { - if existing, exists := m[key]; exists { - m[key] = max(existing, value) - } else { - m[key] = value - } -} - func calcUsageRatio(usedMemory int64, maxMemory int64) float64 { if maxMemory <= 0 { return 0 diff --git a/downstreamadapter/sink/blackhole/sink.go b/downstreamadapter/sink/blackhole/sink.go index b2dc727cdc..c6dc52dcda 100644 --- a/downstreamadapter/sink/blackhole/sink.go +++ b/downstreamadapter/sink/blackhole/sink.go @@ -42,7 +42,7 @@ func (s *sink) SinkType() common.SinkType { return common.BlackHoleSinkType } -func (s *sink) SetTableSchemaStore(tableSchemaStore *commonEvent.TableSchemaStore) { +func (s *sink) SetTableSchemaStore(_ *commonEvent.TableSchemaStore) { } func (s *sink) AddDMLEvent(event *commonEvent.DMLEvent) { diff --git a/downstreamadapter/sink/mysql/causality/node.go b/downstreamadapter/sink/mysql/causality/node.go index d84f63c337..58d71cb82a 100644 --- a/downstreamadapter/sink/mysql/causality/node.go +++ b/downstreamadapter/sink/mysql/causality/node.go @@ -253,27 +253,6 @@ func (n *Node) getOrCreateDependers() *btree.BTreeG[*Node] { return n.dependers } -// dependerCount returns the number of dependers the node has. -// NOTE: dependerCount is used for unit tests only. -func (n *Node) dependerCount() int { - n.mu.Lock() - defer n.mu.Unlock() - - if n.dependers == nil { - return 0 - } - return n.dependers.Len() -} - -// assignedWorkerID returns the cache ID that the node has been assigned to. -// NOTE: assignedWorkerID is used for unit tests only. -func (n *Node) assignedWorkerID() cacheID { - n.mu.Lock() - defer n.mu.Unlock() - - return n.assignedTo -} - func genNextNodeID() int64 { return nextNodeID.Add(1) } diff --git a/downstreamadapter/sink/pulsar/ddl_producer.go b/downstreamadapter/sink/pulsar/ddl_producer.go index a688dbc6dd..2bb33a3b08 100644 --- a/downstreamadapter/sink/pulsar/ddl_producer.go +++ b/downstreamadapter/sink/pulsar/ddl_producer.go @@ -73,7 +73,7 @@ func newDDLProducers( producerCacheSize = int(*sinkConfig.PulsarConfig.PulsarProducerCacheSize) } - producers, err := lru.NewWithEvict(producerCacheSize, func(key interface{}, value interface{}) { + producers, err := lru.NewWithEvict(producerCacheSize, func(_ interface{}, value interface{}) { // remove producer pulsarProducer, ok := value.(pulsarClient.Producer) if ok && pulsarProducer != nil { diff --git a/downstreamadapter/sink/pulsar/dml_producer.go b/downstreamadapter/sink/pulsar/dml_producer.go index b36d75fcf1..f311ed7935 100644 --- a/downstreamadapter/sink/pulsar/dml_producer.go +++ b/downstreamadapter/sink/pulsar/dml_producer.go @@ -88,7 +88,7 @@ func newDMLProducers( producerCacheSize = int(*comp.config.PulsarProducerCacheSize) } - producers, err := lru.NewWithEvict(producerCacheSize, func(key interface{}, value interface{}) { + producers, err := lru.NewWithEvict(producerCacheSize, func(_ interface{}, value interface{}) { // this is call when lru Remove producer or auto remove producer pulsarProducer, ok := value.(pulsarClient.Producer) if ok && pulsarProducer != nil { @@ -161,7 +161,7 @@ func (p *dmlProducers) asyncSendMessage( // if for stress test record , add count to message callback function producer.SendAsync(ctx, data, - func(id pulsarClient.MessageID, m *pulsarClient.ProducerMessage, err error) { + func(_ pulsarClient.MessageID, m *pulsarClient.ProducerMessage, err error) { // fail if err != nil { e := errors.WrapError(errors.ErrPulsarAsyncSendMessage, err) diff --git a/downstreamadapter/sink/pulsar/helper.go b/downstreamadapter/sink/pulsar/helper.go index acc1cddd38..d2c33430b5 100644 --- a/downstreamadapter/sink/pulsar/helper.go +++ b/downstreamadapter/sink/pulsar/helper.go @@ -106,7 +106,7 @@ func newPulsarSinkComponentWithFactory(ctx context.Context, return pulsarComponent, protocol, errors.Trace(err) } - pulsarComponent.topicManager, err = topicmanager.GetPulsarTopicManagerAndTryCreateTopic(ctx, pulsarComponent.config, topic, pulsarComponent.client) + pulsarComponent.topicManager, err = topicmanager.GetPulsarTopicManagerAndTryCreateTopic(ctx, pulsarComponent.config, pulsarComponent.client) if err != nil { return pulsarComponent, protocol, errors.Trace(err) } diff --git a/downstreamadapter/sink/pulsar/mock_producer.go b/downstreamadapter/sink/pulsar/mock_producer.go index 0cc0fa032c..a71d6a1247 100644 --- a/downstreamadapter/sink/pulsar/mock_producer.go +++ b/downstreamadapter/sink/pulsar/mock_producer.go @@ -52,8 +52,8 @@ func (p *mockProducer) syncBroadcastMessage(ctx context.Context, topic string, m // SyncSendMessage sends a message // partitionNum is not used,pulsar consume all partitions -func (p *mockProducer) syncSendMessage(ctx context.Context, topic string, - message *common.Message, messageType common.MessageType, +func (p *mockProducer) syncSendMessage(_ context.Context, topic string, + message *common.Message, _ common.MessageType, ) error { p.mu.Lock() defer p.mu.Unlock() @@ -66,7 +66,7 @@ func (p *mockProducer) syncSendMessage(ctx context.Context, topic string, } // GetProducerByTopic returns a producer by topic name -func (p *mockProducer) GetProducerByTopic(topicName string) (producer pulsar.Producer, err error) { +func (p *mockProducer) GetProducerByTopic(_ string) (producer pulsar.Producer, err error) { return producer, nil } diff --git a/downstreamadapter/sink/pulsar/sink.go b/downstreamadapter/sink/pulsar/sink.go index 38c3008321..7826ff89d6 100644 --- a/downstreamadapter/sink/pulsar/sink.go +++ b/downstreamadapter/sink/pulsar/sink.go @@ -493,7 +493,7 @@ func (s *sink) batchEncodeRun(ctx context.Context) error { // batch collects a batch of messages from w.msgChan into buffer. // Note: It will block until at least one message is received. -func (s *sink) batch(ctx context.Context, buffer []*commonEvent.MQRowEvent, ticker *time.Ticker) ([]*commonEvent.MQRowEvent, error) { +func (s *sink) batch(ctx context.Context, buffer []*commonEvent.MQRowEvent, _ *time.Ticker) ([]*commonEvent.MQRowEvent, error) { // We need to receive at least one message or be interrupted, // otherwise it will lead to idling. select { diff --git a/downstreamadapter/sink/redo/sink.go b/downstreamadapter/sink/redo/sink.go index 10b81b39b8..420a407b1b 100644 --- a/downstreamadapter/sink/redo/sink.go +++ b/downstreamadapter/sink/redo/sink.go @@ -86,10 +86,20 @@ func New(ctx context.Context, changefeedID common.ChangeFeedID, return } if ddlWriter != nil { - ddlWriter.Close() + if closeErr := ddlWriter.Close(); closeErr != nil && !errors.Is(closeErr, context.Canceled) { + log.Warn("redo: failed to close ddl writer after create failure", + zap.String("keyspace", changefeedID.Keyspace()), + zap.String("changefeed", changefeedID.Name()), + zap.Error(closeErr)) + } } if dmlWriter != nil { - dmlWriter.Close() + if closeErr := dmlWriter.Close(); closeErr != nil && !errors.Is(closeErr, context.Canceled) { + log.Warn("redo: failed to close dml writer after create failure", + zap.String("keyspace", changefeedID.Keyspace()), + zap.String("changefeed", changefeedID.Name()), + zap.Error(closeErr)) + } } }() @@ -197,7 +207,7 @@ func (s *Sink) Close() { start := time.Now() s.logBuffer.Close() if s.ddlWriter != nil { - if err := s.ddlWriter.Close(); err != nil && errors.Cause(err) != context.Canceled { + if err := s.ddlWriter.Close(); err != nil && !errors.Is(err, context.Canceled) { log.Error("redo sink fails to close ddl writer", zap.String("keyspace", s.changefeedID.Keyspace()), zap.String("changefeed", s.changefeedID.Name()), @@ -205,7 +215,7 @@ func (s *Sink) Close() { } } if s.dmlWriter != nil { - if err := s.dmlWriter.Close(); err != nil && errors.Cause(err) != context.Canceled { + if err := s.dmlWriter.Close(); err != nil && !errors.Is(err, context.Canceled) { log.Error("redo sink fails to close dml writer", zap.String("keyspace", s.changefeedID.Keyspace()), zap.String("changefeed", s.changefeedID.Name()), diff --git a/downstreamadapter/sink/topicmanager/kafka_topic_manager.go b/downstreamadapter/sink/topicmanager/kafka_topic_manager.go index c6963573d8..5cf7f6049a 100644 --- a/downstreamadapter/sink/topicmanager/kafka_topic_manager.go +++ b/downstreamadapter/sink/topicmanager/kafka_topic_manager.go @@ -19,10 +19,9 @@ import ( "sync" "time" - "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/ticdc/pkg/common" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/retry" "github.com/pingcap/ticdc/pkg/sink/kafka" "go.uber.org/zap" @@ -66,7 +65,7 @@ func GetTopicManagerAndTryCreateTopic( ) if _, err := topicManager.CreateTopicAndWaitUntilVisible(ctx, topic); err != nil { - return nil, cerror.WrapError(cerror.ErrKafkaCreateTopic, err) + return nil, errors.WrapError(errors.ErrKafkaCreateTopic, err) } return topicManager, nil @@ -130,7 +129,7 @@ func (m *kafkaTopicManager) backgroundRefreshMeta(ctx context.Context) { case <-m.metaRefreshTicker.C: // We ignore the error here, because the error may be caused by the // network problem, and we can try to get the metadata next time. - topicPartitionNums, _ := m.fetchAllTopicsPartitionsNum(ctx) + topicPartitionNums, _ := m.fetchAllTopicsPartitionsNum() for topic, partitionNum := range topicPartitionNums { m.tryUpdatePartitionsAndLogging(topic, partitionNum) } @@ -169,11 +168,9 @@ func (m *kafkaTopicManager) tryUpdatePartitionsAndLogging(topic string, partitio // The error returned by this method could be a transient error that is fixable by the underlying logic. // When handling this error, please be cautious. // If you simply throw the error to the caller, it may impact the robustness of your program. -func (m *kafkaTopicManager) fetchAllTopicsPartitionsNum( - ctx context.Context, -) (map[string]int32, error) { +func (m *kafkaTopicManager) fetchAllTopicsPartitionsNum() (map[string]int32, error) { var topics []string - m.topics.Range(func(key, value any) bool { + m.topics.Range(func(key, _ any) bool { topics = append(topics, key.(string)) return true }) @@ -244,11 +241,11 @@ func (m *kafkaTopicManager) waitUntilTopicVisible( // createTopic creates a topic with the given name // and returns the number of partitions. func (m *kafkaTopicManager) createTopic( - ctx context.Context, + _ context.Context, topicName string, ) (int32, error) { if !m.cfg.AutoCreate { - return 0, cerror.ErrKafkaInvalidConfig.GenWithStack( + return 0, errors.ErrKafkaInvalidConfig.GenWithStack( fmt.Sprintf("`auto-create-topic` is false, "+ "and %s not found", topicName)) } @@ -270,7 +267,7 @@ func (m *kafkaTopicManager) createTopic( zap.Error(err), zap.Duration("duration", time.Since(start)), ) - return 0, cerror.WrapError(cerror.ErrKafkaCreateTopic, err) + return 0, errors.WrapError(errors.ErrKafkaCreateTopic, err) } log.Info( diff --git a/downstreamadapter/sink/topicmanager/pulsar_topic_manager.go b/downstreamadapter/sink/topicmanager/pulsar_topic_manager.go index c2d02ba6e2..0158a072e2 100644 --- a/downstreamadapter/sink/topicmanager/pulsar_topic_manager.go +++ b/downstreamadapter/sink/topicmanager/pulsar_topic_manager.go @@ -30,9 +30,8 @@ type pulsarTopicManager struct { // GetPulsarTopicManagerAndTryCreateTopic returns the topic manager and try to create the topic. func GetPulsarTopicManagerAndTryCreateTopic( - ctx context.Context, + _ context.Context, cfg *config.PulsarConfig, - topic string, client pulsar.Client, ) (TopicManager, error) { topicManager := newPulsarTopicManager(cfg, client) @@ -53,7 +52,7 @@ func newPulsarTopicManager( // GetPartitionNum always return 1 because we pass a message key to pulsar producer, // and pulsar producer will hash the key to a partition. // This method is only used to meet the requirement of mq sink's interface. -func (m *pulsarTopicManager) GetPartitionNum(ctx context.Context, topic string) (int32, error) { +func (m *pulsarTopicManager) GetPartitionNum(_ context.Context, _ string) (int32, error) { return 1, nil } diff --git a/downstreamadapter/sink/topicmanager/pulsar_topic_manager_mock.go b/downstreamadapter/sink/topicmanager/pulsar_topic_manager_mock.go index 37f47b045f..9ba0705a6a 100644 --- a/downstreamadapter/sink/topicmanager/pulsar_topic_manager_mock.go +++ b/downstreamadapter/sink/topicmanager/pulsar_topic_manager_mock.go @@ -15,9 +15,6 @@ package topicmanager import ( "context" - - "github.com/apache/pulsar-client-go/pulsar" - "github.com/pingcap/ticdc/pkg/config" ) // pulsarTopicManager is a manager for pulsar topics. @@ -26,16 +23,13 @@ type pulsarTopicManagerMock struct { } // NewMockPulsarTopicManager creates a new topic manager. -func NewMockPulsarTopicManager( - cfg *config.PulsarConfig, - client pulsar.Client, -) (TopicManager, error) { +func NewMockPulsarTopicManager() (TopicManager, error) { mgr := &pulsarTopicManagerMock{} return mgr, nil } // GetPartitionNum spend more time,but no use. // mock 3 partitions -func (m *pulsarTopicManagerMock) GetPartitionNum(ctx context.Context, topic string) (int32, error) { +func (m *pulsarTopicManagerMock) GetPartitionNum(_ context.Context, _ string) (int32, error) { return 3, nil } diff --git a/downstreamadapter/sink/topicmanager/pulsar_topic_manager_test.go b/downstreamadapter/sink/topicmanager/pulsar_topic_manager_test.go index 9d6aa33d7c..e5061e4392 100644 --- a/downstreamadapter/sink/topicmanager/pulsar_topic_manager_test.go +++ b/downstreamadapter/sink/topicmanager/pulsar_topic_manager_test.go @@ -42,8 +42,6 @@ func newPulsarConfig(t *testing.T) (*config.PulsarConfig, *url.URL) { func TestGetPartitionNumMock(t *testing.T) { t.Parallel() - cfg, _ := newPulsarConfig(t) - replicaConfig := config.GetDefaultReplicaConfig() replicaConfig.Sink = &config.SinkConfig{ Protocol: util.AddressOf("canal-json"), @@ -52,7 +50,7 @@ func TestGetPartitionNumMock(t *testing.T) { ctx := context.Background() ctx = context.WithValue(ctx, "testing.T", t) - pm, err := NewMockPulsarTopicManager(cfg, nil) + pm, err := NewMockPulsarTopicManager() require.NoError(t, err) require.NotNil(t, pm) diff --git a/downstreamadapter/syncpoint/sync_point.go b/downstreamadapter/syncpoint/sync_point.go index 1797a855fa..14cbd9be4b 100644 --- a/downstreamadapter/syncpoint/sync_point.go +++ b/downstreamadapter/syncpoint/sync_point.go @@ -31,9 +31,9 @@ func CalculateStartSyncPointTs(startTs uint64, syncPointInterval time.Duration, } k := int64(oracle.GetTimeFromTS(startTs).Sub(time.Unix(0, 0)) / syncPointInterval) if oracle.GetTimeFromTS(startTs).Sub(time.Unix(0, 0))%syncPointInterval != 0 || oracle.ExtractLogical(startTs) != 0 { - k += 1 + k++ } else if skipSyncpointAtStartTs { - k += 1 + k++ } return oracle.GoTimeToTS(time.Unix(0, 0).Add(time.Duration(int64(syncPointInterval) * k))) } diff --git a/logservice/coordinator/coordinator.go b/logservice/coordinator/coordinator.go index 4bf809dbf1..d557b2fb12 100644 --- a/logservice/coordinator/coordinator.go +++ b/logservice/coordinator/coordinator.go @@ -182,7 +182,13 @@ func (c *logCoordinator) sendResolvedTsToCoordinator(id node.ID, changefeedID co ResolvedTs: resolvedTs, }, ) - c.messageCenter.SendEvent(msg) + if err := c.messageCenter.SendEvent(msg); err != nil { + log.Warn("failed to send resolved ts to coordinator", + zap.Stringer("target", id), + zap.String("changefeed", changefeedID.DisplayName.String()), + zap.Uint64("resolvedTs", resolvedTs), + zap.Error(err)) + } } func (c *logCoordinator) handleNodeChange(allNodes map[node.ID]*node.Info) { diff --git a/logservice/eventstore/event_store.go b/logservice/eventstore/event_store.go index e904b2057b..8bc145970a 100644 --- a/logservice/eventstore/event_store.go +++ b/logservice/eventstore/event_store.go @@ -456,7 +456,7 @@ func (e *eventStore) Close(_ context.Context) error { } func (e *eventStore) RegisterDispatcher( - changefeedID common.ChangeFeedID, + _ common.ChangeFeedID, dispatcherID common.DispatcherID, dispatcherSpan *heartbeatpb.TableSpan, startTs uint64, diff --git a/logservice/logpuller/errors.go b/logservice/logpuller/errors.go index eb25ad7059..ad3c48a093 100644 --- a/logservice/logpuller/errors.go +++ b/logservice/logpuller/errors.go @@ -16,13 +16,10 @@ package logpuller import ( "fmt" - "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/cdcpb" "github.com/tikv/client-go/v2/tikv" ) -var errUnreachable = errors.New("kv client unreachable error") - type eventError struct { err *cdcpb.Error } diff --git a/logservice/logpuller/region_event_handler.go b/logservice/logpuller/region_event_handler.go index 0de2306c47..58673733cf 100644 --- a/logservice/logpuller/region_event_handler.go +++ b/logservice/logpuller/region_event_handler.go @@ -169,7 +169,7 @@ func (h *regionEventHandler) GetSize(event regionEvent) int { return event.getSize() } -func (h *regionEventHandler) GetArea(path SubscriptionID, dest *subscribedSpan) int { +func (h *regionEventHandler) GetArea(_ SubscriptionID, _ *subscribedSpan) int { return 0 } @@ -198,7 +198,7 @@ func (h *regionEventHandler) GetTimestamp(event regionEvent) dynstream.Timestamp return dynstream.Timestamp(event.resolvedTs) } } -func (h *regionEventHandler) IsPaused(event regionEvent) bool { return false } +func (h *regionEventHandler) IsPaused(_ regionEvent) bool { return false } func (h *regionEventHandler) GetType(event regionEvent) dynstream.EventType { if event.entries != nil || event.resolvedTs != 0 { diff --git a/logservice/logpuller/subscription_client.go b/logservice/logpuller/subscription_client.go index 105db5d73d..84c3f782ff 100644 --- a/logservice/logpuller/subscription_client.go +++ b/logservice/logpuller/subscription_client.go @@ -19,7 +19,6 @@ import ( "sync/atomic" "time" - "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" "github.com/pingcap/ticdc/heartbeatpb" @@ -28,7 +27,7 @@ import ( "github.com/pingcap/ticdc/pkg/common" appcontext "github.com/pingcap/ticdc/pkg/common/context" "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/metrics" "github.com/pingcap/ticdc/pkg/pdutil" "github.com/pingcap/ticdc/pkg/security" @@ -319,7 +318,7 @@ func (s *subscriptionClient) updateMetrics(ctx context.Context) error { } pendingRegionReqCount := 0 - s.stores.Range(func(key, value any) bool { + s.stores.Range(func(_, value any) bool { store := value.(*requestedStore) store.requestWorkers.RLock() for _, worker := range store.requestWorkers.s { @@ -581,7 +580,7 @@ func (s *subscriptionClient) handleRegions(ctx context.Context, eg *errgroup.Gro } defer func() { - s.stores.Range(func(key, value any) bool { + s.stores.Range(func(_, value any) bool { rs := value.(*requestedStore) rs.requestWorkers.RLock() @@ -855,6 +854,7 @@ func (s *subscriptionClient) doHandleError(ctx context.Context, errInfo regionEr zap.Error(err)) } + //nolint:errorlint // converting large type switch to errors.As is a significant refactor switch eerr := err.(type) { case *eventError: innerErr := eerr.err @@ -890,10 +890,10 @@ func (s *subscriptionClient) doHandleError(ctx context.Context, errInfo regionEr return errors.New("duplicate request") } if compatibility := innerErr.GetCompatibility(); compatibility != nil { - return cerror.ErrVersionIncompatible.GenWithStackByArgs(compatibility) + return errors.ErrVersionIncompatible.GenWithStackByArgs(compatibility) } if mismatch := innerErr.GetClusterIdMismatch(); mismatch != nil { - return cerror.ErrClusterIDMismatch.GenWithStackByArgs(mismatch.Current, mismatch.Request) + return errors.ErrClusterIDMismatch.GenWithStackByArgs(mismatch.Current, mismatch.Request) } log.Warn("empty or unknown cdc error", @@ -1046,7 +1046,7 @@ func (s *subscriptionClient) logSlowRegions(ctx context.Context) error { zap.Any("slowRegion", attr.SlowestRegion)) } } else if currTime.Sub(attr.SlowestRegion.Created) > 10*time.Minute { - slowInitializeRegion += 1 + slowInitializeRegion++ log.Info("subscription client initializes a region too slow", zap.Uint64("subscriptionID", uint64(subscriptionID)), zap.Any("slowRegion", attr.SlowestRegion)) diff --git a/maintainer/barrier_event.go b/maintainer/barrier_event.go index cf81e05a5e..d603d4c732 100644 --- a/maintainer/barrier_event.go +++ b/maintainer/barrier_event.go @@ -742,8 +742,3 @@ func getAllNodes(nodeManager *watcher.NodeManager) []node.ID { } return nodes } - -// for test -func (be *BarrierEvent) setLastResendTime(time time.Time) { - be.lastResendTime = time -} diff --git a/maintainer/replica/default_span_split_checker.go b/maintainer/replica/default_span_split_checker.go index 351292e897..8b849fdd9a 100644 --- a/maintainer/replica/default_span_split_checker.go +++ b/maintainer/replica/default_span_split_checker.go @@ -29,10 +29,7 @@ import ( "go.uber.org/zap" ) -var ( - trafficScoreThreshold = 3 - regionScoreThreshold = 3 -) +var trafficScoreThreshold = 3 // defaultSpanSplitChecker is used to check whether spans in the default group need to be split // based on multiple thresholds including write traffic and region count. diff --git a/maintainer/testutil/test_util.go b/maintainer/testutil/test_util.go index 05ea81d481..c7a7f0d85a 100644 --- a/maintainer/testutil/test_util.go +++ b/maintainer/testutil/test_util.go @@ -92,7 +92,7 @@ func (m *MockCache) SetRegions(key string, regions []*tikv.Region) { } func (m *MockCache) LoadRegionsInKeyRange( - bo *tikv.Backoffer, startKey, endKey []byte, + _ *tikv.Backoffer, startKey, endKey []byte, ) (regions []*tikv.Region, err error) { if m.err != nil { return nil, m.err diff --git a/pkg/check/active_active_tso_indexes.go b/pkg/check/active_active_tso_indexes.go index b662da5b9a..9587262543 100644 --- a/pkg/check/active_active_tso_indexes.go +++ b/pkg/check/active_active_tso_indexes.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/ticdc/pkg/config" "github.com/pingcap/ticdc/pkg/errors" - cerrors "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/pdutil" pd "github.com/tikv/pd/client" pdhttp "github.com/tikv/pd/client/http" @@ -55,7 +54,7 @@ func ValidateActiveActiveTSOIndexes( changefeedCfg *config.ChangefeedConfig, ) error { if changefeedCfg == nil { - return cerrors.ErrActiveActiveTSOIndexIncompatible.GenWithStackByArgs("changefeed config is nil") + return errors.ErrActiveActiveTSOIndexIncompatible.GenWithStackByArgs("changefeed config is nil") } if !changefeedCfg.EnableActiveActive { return nil @@ -71,8 +70,8 @@ func ValidateActiveActiveTSOIndexes( downUnique, downMax, err := getDownstreamTSOIndexes(ctx, changefeedCfg, sinkURI) if err != nil { - return cerrors.WrapError( - cerrors.ErrActiveActiveTSOIndexIncompatible, + return errors.WrapError( + errors.ErrActiveActiveTSOIndexIncompatible, err, "failed to read downstream tso index config", ) @@ -80,8 +79,8 @@ func ValidateActiveActiveTSOIndexes( upUnique, upMax, err := getUpstreamTSOIndexes(ctx, upPD) if err != nil { - return cerrors.WrapError( - cerrors.ErrActiveActiveTSOIndexIncompatible, + return errors.WrapError( + errors.ErrActiveActiveTSOIndexIncompatible, err, fmt.Sprintf("failed to read upstream tso index config, downstream unique=%d, downstream max=%d", downUnique, downMax), @@ -92,13 +91,13 @@ func ValidateActiveActiveTSOIndexes( // downstream to avoid TSO collisions, while `tso-max-index` must match to guarantee // the same logical index range. if upUnique == downUnique { - return cerrors.ErrActiveActiveTSOIndexIncompatible.GenWithStackByArgs( + return errors.ErrActiveActiveTSOIndexIncompatible.GenWithStackByArgs( fmt.Sprintf("active active tso index mismatch, upstream and downstream share the same tso-unique-index=%d, upstream max=%d, downstream max=%d", upUnique, upMax, downMax), ) } if upMax != downMax { - return cerrors.ErrActiveActiveTSOIndexIncompatible.GenWithStackByArgs( + return errors.ErrActiveActiveTSOIndexIncompatible.GenWithStackByArgs( fmt.Sprintf("active active tso index mismatch, upstream unique=%d, upstream max=%d, downstream unique=%d, downstream max=%d", upUnique, upMax, downUnique, downMax), ) @@ -121,18 +120,18 @@ func getDownstreamTSOIndexes( sinkURI *url.URL, ) (unique int64, max int64, err error) { if changefeedCfg == nil { - return 0, 0, cerrors.New("changefeed config is nil") + return 0, 0, errors.New("changefeed config is nil") } mysqlCfg, db, err := newMySQLConfigAndDBFn(ctx, changefeedCfg.ChangefeedID, sinkURI, changefeedCfg) if err != nil { - return 0, 0, cerrors.Trace(err) + return 0, 0, errors.Trace(err) } defer func() { _ = db.Close() }() readTimeout, err := time.ParseDuration(mysqlCfg.ReadTimeout) if err != nil { - return 0, 0, cerrors.Trace(err) + return 0, 0, errors.Trace(err) } // Bound the downstream query by the sink read timeout to keep the validation @@ -142,7 +141,7 @@ func getDownstreamTSOIndexes( rows, err := db.QueryContext(queryCtx, showPDConfigQuery) if err != nil { - return 0, 0, cerrors.Trace(err) + return 0, 0, errors.Trace(err) } defer func() { _ = rows.Close() }() @@ -157,13 +156,13 @@ func getDownstreamTSOIndexes( // Columns: Type | Instance | Name | Value var typ, instance, name, value string if err := rows.Scan(&typ, &instance, &name, &value); err != nil { - return 0, 0, cerrors.Trace(err) + return 0, 0, errors.Trace(err) } switch name { case pdTSOUniqueIndexKey: parsed, err := strconv.ParseInt(value, 10, 64) if err != nil { - return 0, 0, cerrors.Trace(err) + return 0, 0, errors.Trace(err) } if !uniqueSet { unique = parsed @@ -171,12 +170,12 @@ func getDownstreamTSOIndexes( continue } if unique != parsed { - return 0, 0, cerrors.New("downstream TiDB reports inconsistent tso-unique-index across instances") + return 0, 0, errors.New("downstream TiDB reports inconsistent tso-unique-index across instances") } case pdTSOMaxIndexKey: parsed, err := strconv.ParseInt(value, 10, 64) if err != nil { - return 0, 0, cerrors.Trace(err) + return 0, 0, errors.Trace(err) } if !maxSet { max = parsed @@ -184,20 +183,20 @@ func getDownstreamTSOIndexes( continue } if max != parsed { - return 0, 0, cerrors.New("downstream TiDB reports inconsistent tso-max-index across instances") + return 0, 0, errors.New("downstream TiDB reports inconsistent tso-max-index across instances") } default: } } if err := rows.Err(); err != nil { - return 0, 0, cerrors.Trace(err) + return 0, 0, errors.Trace(err) } if !uniqueSet { - return 0, 0, cerrors.Errorf("downstream TiDB does not report %s", pdTSOUniqueIndexKey) + return 0, 0, errors.Errorf("downstream TiDB does not report %s", pdTSOUniqueIndexKey) } if !maxSet { - return 0, 0, cerrors.Errorf("downstream TiDB does not report %s", pdTSOMaxIndexKey) + return 0, 0, errors.Errorf("downstream TiDB does not report %s", pdTSOMaxIndexKey) } return unique, max, nil } @@ -215,27 +214,27 @@ func getUpstreamTSOIndexes( upPD pd.Client, ) (unique int64, max int64, err error) { if upPD == nil { - return 0, 0, cerrors.New("pd client is nil") + return 0, 0, errors.New("pd client is nil") } httpClient, err := newPDHTTPClientFn(upPD) if err != nil { - return 0, 0, cerrors.Trace(err) + return 0, 0, errors.Trace(err) } defer httpClient.Close() cfg, err := httpClient.GetConfig(ctx) if err != nil { - return 0, 0, cerrors.Trace(err) + return 0, 0, errors.Trace(err) } unique, err = parsePDConfigInt64(cfg, pdTSOUniqueIndexKey) if err != nil { - return 0, 0, cerrors.Trace(err) + return 0, 0, errors.Trace(err) } max, err = parsePDConfigInt64(cfg, pdTSOMaxIndexKey) if err != nil { - return 0, 0, cerrors.Trace(err) + return 0, 0, errors.Trace(err) } return unique, max, nil } @@ -243,7 +242,7 @@ func getUpstreamTSOIndexes( func parsePDConfigInt64(cfg map[string]any, key string) (int64, error) { v, ok := cfg[key] if !ok { - return 0, cerrors.Errorf("pd config key not found: %s", key) + return 0, errors.Errorf("pd config key not found: %s", key) } // PD stores `tso-unique-index` and `tso-max-index` as int64 values. @@ -263,16 +262,16 @@ func parsePDConfigInt64(cfg map[string]any, key string) (int64, error) { // on overflow. const maxExactIntInFloat64 = float64(1 << 53) if math.IsNaN(x) || math.IsInf(x, 0) { - return 0, cerrors.New("value is not a finite number") + return 0, errors.New("value is not a finite number") } if math.Trunc(x) != x { - return 0, cerrors.New("value is not an integer") + return 0, errors.New("value is not an integer") } if x > maxExactIntInFloat64 || x < -maxExactIntInFloat64 { - return 0, cerrors.Errorf("value for %s exceeds exact integer range for float64", key) + return 0, errors.Errorf("value for %s exceeds exact integer range for float64", key) } return int64(x), nil default: - return 0, cerrors.Errorf("unexpected value type for %s: %T", key, v) + return 0, errors.Errorf("unexpected value type for %s: %T", key, v) } } diff --git a/pkg/common/event/active_active.go b/pkg/common/event/active_active.go index 2ee11c2ffb..2b91c2e616 100644 --- a/pkg/common/event/active_active.go +++ b/pkg/common/event/active_active.go @@ -18,8 +18,6 @@ import ( "github.com/pingcap/ticdc/pkg/common" "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/integrity" - "github.com/pingcap/tidb/pkg/parser/mysql" - tidbTypes "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "go.uber.org/zap" ) @@ -116,52 +114,6 @@ func isSoftDeleteTransition(preRow, row chunk.Row, offset int) bool { return preRow.IsNull(offset) && !row.IsNull(offset) } -// getSoftDeleteTimeColumnIndex returns the column offset for `_tidb_softdelete_time` after validating -// its semantics. It reports schema issues via `handleError` and returns ok=false in that case. -// -// The soft-delete transition detection relies on the invariant that `_tidb_softdelete_time` is -// defined as `TIMESTAMP(6) NULL`, and NULL is the only representation of "not deleted". -func getSoftDeleteTimeColumnIndex(event *DMLEvent, handleError func(error)) (idx int, ok bool) { - tableInfo := event.TableInfo - colInfo, ok := tableInfo.GetColumnInfoByName(SoftDeleteTimeColumn) - if !ok { - handleError(errors.Errorf( - "dispatcher %s table %s.%s missing required column %s", - event.DispatcherID.String(), - tableInfo.GetSchemaName(), - tableInfo.GetTableName(), - SoftDeleteTimeColumn, - )) - return 0, false - } - offset, ok := tableInfo.GetColumnOffsetByName(SoftDeleteTimeColumn) - if !ok { - handleError(errors.Errorf( - "dispatcher %s table %s.%s missing required column offset %s", - event.DispatcherID.String(), - tableInfo.GetSchemaName(), - tableInfo.GetTableName(), - SoftDeleteTimeColumn, - )) - return 0, false - } - notNull := mysql.HasNotNullFlag(colInfo.GetFlag()) - if colInfo.GetType() != mysql.TypeTimestamp || colInfo.FieldType.GetDecimal() != tidbTypes.MaxFsp || notNull { - handleError(errors.Errorf( - "dispatcher %s table %s.%s invalid column %s, expect TIMESTAMP(6) NULL, got type %d fsp %d notNull %t", - event.DispatcherID.String(), - tableInfo.GetSchemaName(), - tableInfo.GetTableName(), - SoftDeleteTimeColumn, - colInfo.GetType(), - colInfo.FieldType.GetDecimal(), - notNull, - )) - return 0, false - } - return offset, true -} - // getSoftDeleteTimeColumnOffset returns the column offset for `_tidb_softdelete_time` without validating // its type semantics. It reports schema issues via `handleError` and returns ok=false in that case. func getSoftDeleteTimeColumnOffset(event *DMLEvent, handleError func(error)) (idx int, ok bool) { diff --git a/pkg/common/event/dml_event.go b/pkg/common/event/dml_event.go index cdc7c7d7fa..d995adfdf0 100644 --- a/pkg/common/event/dml_event.go +++ b/pkg/common/event/dml_event.go @@ -628,7 +628,7 @@ func (t *DMLEvent) AppendRow(raw *common.RawKVEntry, copy(keyCopy, raw.Key) t.RowKeys = append(t.RowKeys, keyCopy) } - t.Length += 1 + t.Length++ t.ApproximateSize += raw.GetSize() if checksum != nil { t.Checksum = append(t.Checksum, checksum) diff --git a/pkg/common/log_coordinator.go b/pkg/common/log_coordinator.go index 21302ee338..d0575bed05 100644 --- a/pkg/common/log_coordinator.go +++ b/pkg/common/log_coordinator.go @@ -19,6 +19,6 @@ func (l *LogCoordinatorBroadcastRequest) Marshal() ([]byte, error) { return nil, nil } -func (l *LogCoordinatorBroadcastRequest) Unmarshal(data []byte) error { +func (l *LogCoordinatorBroadcastRequest) Unmarshal(_ []byte) error { return nil } diff --git a/pkg/diff/checkpoint.go b/pkg/diff/checkpoint.go index 5f9e63c72b..f00e9d1ae3 100644 --- a/pkg/diff/checkpoint.go +++ b/pkg/diff/checkpoint.go @@ -22,8 +22,8 @@ import ( "sync" "time" - "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/tidb/pkg/util/dbutil" "go.uber.org/zap" ) @@ -166,7 +166,9 @@ func loadChunks(ctx context.Context, db *sql.DB, instanceID, schema, table strin if err != nil { return nil, err } - defer rows.Close() + defer func() { + _ = rows.Close() + }() for rows.Next() { fields, err1 := dbutil.ScanRow(rows) @@ -327,7 +329,9 @@ func loadFromCheckPoint(ctx context.Context, db *sql.DB, schema, table, configHa if err != nil { return false, errors.Trace(err) } - defer rows.Close() + defer func() { + _ = rows.Close() + }() var state, cfgHash sql.NullString diff --git a/pkg/diff/diff.go b/pkg/diff/diff.go index 29d1d6e49c..a53121f5ae 100644 --- a/pkg/diff/diff.go +++ b/pkg/diff/diff.go @@ -27,9 +27,9 @@ import ( "sync/atomic" "time" - "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/util" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/util/dbutil" @@ -580,7 +580,9 @@ func (t *TableDiff) compareRows(ctx context.Context, chunk *ChunkRange) (bool, e if targetRows.Err() != nil { return false, errors.Trace(targetRows.Err()) } - defer targetRows.Close() + defer func() { + _ = targetRows.Close() + }() for i, sourceTable := range t.SourceTables { rows, _, err := getChunkRows(ctx, sourceTable.Conn, sourceTable.Schema, sourceTable.Table, sourceTable.info, @@ -591,7 +593,9 @@ func (t *TableDiff) compareRows(ctx context.Context, chunk *ChunkRange) (bool, e if rows.Err() != nil { return false, errors.Trace(rows.Err()) } - defer rows.Close() + defer func() { + _ = rows.Close() + }() sourceRows[i] = rows sourceHaveData[i] = false diff --git a/pkg/encryption/tikv_http_client.go b/pkg/encryption/tikv_http_client.go index 16b74d92fb..d2c9d971f1 100644 --- a/pkg/encryption/tikv_http_client.go +++ b/pkg/encryption/tikv_http_client.go @@ -22,9 +22,8 @@ import ( "time" "github.com/gogo/protobuf/proto" - "github.com/pingcap/errors" "github.com/pingcap/log" - cerrors "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/httputil" "github.com/pingcap/ticdc/pkg/security" "github.com/pingcap/tidb/pkg/util/engine" @@ -83,7 +82,7 @@ func (c *tikvEncryptionHTTPClient) GetKeyspaceEncryptionMeta(ctx context.Context if err == nil { return meta, nil } - if cerrors.ErrEncryptionMetaNotFound.Equal(err) { + if errors.ErrEncryptionMetaNotFound.Equal(err) { lastErr = err continue } @@ -91,7 +90,7 @@ func (c *tikvEncryptionHTTPClient) GetKeyspaceEncryptionMeta(ctx context.Context } if lastErr == nil { - lastErr = cerrors.ErrEncryptionMetaNotFound + lastErr = errors.ErrEncryptionMetaNotFound } return nil, lastErr } @@ -113,7 +112,9 @@ func (c *tikvEncryptionHTTPClient) getEncryptionMetaFromStore(ctx context.Contex zap.Error(err)) return nil, errors.Trace(err) } - defer resp.Body.Close() + defer func() { + _ = resp.Body.Close() + }() body, err := io.ReadAll(resp.Body) if err != nil { @@ -133,7 +134,7 @@ func (c *tikvEncryptionHTTPClient) getEncryptionMetaFromStore(ctx context.Contex zap.String("statusAddr", statusAddr), zap.Uint32("keyspaceID", keyspaceID), zap.String("url", storeURL)) - return nil, cerrors.ErrEncryptionMetaNotFound + return nil, errors.ErrEncryptionMetaNotFound } if resp.StatusCode < 200 || resp.StatusCode >= 300 { log.Warn("unexpected encryption meta response status", @@ -285,7 +286,7 @@ func decodeEncryptionMetaResponseFromProtobuf(body []byte) (*encryptionMetaRespo return nil, errors.Trace(err) } if metaPB.Current == nil && metaPB.MasterKey == nil && len(metaPB.DataKeys) == 0 && len(metaPB.History) == 0 && metaPB.KeyspaceId == 0 { - return nil, cerrors.ErrDecodeFailed.GenWithStackByArgs("protobuf payload does not contain encryption meta fields") + return nil, errors.ErrDecodeFailed.GenWithStackByArgs("protobuf payload does not contain encryption meta fields") } return metaPB.toEncryptionMetaResponse(), nil } @@ -342,7 +343,7 @@ func (r *encryptionMetaResponse) toEncryptionMeta() (*EncryptionMeta, error) { if r.Current.DataKeyId == 0 { log.Warn("invalid encryption meta from TiKV: current data key ID is empty", zap.Uint32("metaKeyspaceID", r.KeyspaceId)) - return nil, cerrors.ErrEncryptionMetaNotFound + return nil, errors.ErrEncryptionMetaNotFound } version := byte(r.Current.DataKeyId & 0xFF) @@ -350,7 +351,7 @@ func (r *encryptionMetaResponse) toEncryptionMeta() (*EncryptionMeta, error) { log.Warn("invalid encryption meta from TiKV: version must be non-zero", zap.Uint32("metaKeyspaceID", r.KeyspaceId), zap.Uint32("currentDataKeyID", r.Current.DataKeyId)) - return nil, cerrors.ErrEncryptionFailed.GenWithStackByArgs("version must be non-zero") + return nil, errors.ErrEncryptionFailed.GenWithStackByArgs("version must be non-zero") } dataKeys := make(map[uint32]*DataKey, len(r.DataKeys)) @@ -363,7 +364,7 @@ func (r *encryptionMetaResponse) toEncryptionMeta() (*EncryptionMeta, error) { zap.Uint32("metaKeyspaceID", r.KeyspaceId), zap.Uint32("currentDataKeyID", r.Current.DataKeyId), zap.Int("dataKeyCount", len(dataKeys))) - return nil, cerrors.ErrDataKeyNotFound.GenWithStackByArgs("current data key not found") + return nil, errors.ErrDataKeyNotFound.GenWithStackByArgs("current data key not found") } history := make([]*EncryptionEpoch, 0, len(r.History)) diff --git a/pkg/errors/helper.go b/pkg/errors/helper.go index cb522aa5bb..8b1276f32e 100644 --- a/pkg/errors/helper.go +++ b/pkg/errors/helper.go @@ -41,8 +41,7 @@ func IsRetryableError(err error) bool { return false } - switch errors.Cause(err) { - case context.Canceled, context.DeadlineExceeded: + if Is(Cause(err), context.Canceled) || Is(Cause(err), context.DeadlineExceeded) { return false } return true @@ -51,9 +50,8 @@ func IsRetryableError(err error) bool { // IsConnectionError tells whether this error should reconnect to Database. // Return true also means caller can retry sql safely. func IsConnectionError(err error) bool { - err = errors.Cause(err) - switch err { - case driver.ErrBadConn, tmysql.ErrBadConn, gmysql.ErrBadConn: + err = Cause(err) + if Is(err, driver.ErrBadConn) || Is(err, tmysql.ErrBadConn) || Is(err, gmysql.ErrBadConn) { return true } return false @@ -63,7 +61,7 @@ func IsConnectionError(err error) bool { func IsUnretryableConnectionError(err error) bool { // Can't ensure whether the last write has reached the downstream or not. // If the last write isn't idempotent, retry it may cause problems. - return errors.Cause(err) == dmysql.ErrInvalidConn + return Is(Cause(err), dmysql.ErrInvalidConn) } // ChangeFeedGCFastFailError is read only. diff --git a/pkg/errors/reexport.go b/pkg/errors/reexport.go index 01c1beb4b0..5c05119722 100644 --- a/pkg/errors/reexport.go +++ b/pkg/errors/reexport.go @@ -19,23 +19,30 @@ import ( perrors "github.com/pingcap/errors" ) +type ( + Error = perrors.Error + RFCErrorCode = perrors.RFCErrorCode + ErrCode = perrors.ErrCode +) + var ( - // Is is a shortcut for errors.Is. - Is = errors.Is - // As is a shortcut for errors.As. - As = errors.As - // New is a shortcut for github.com/pingcap/errors.New. - New = perrors.New - // Errorf is a shortcut for github.com/pingcap/errors.Errorf. - Errorf = perrors.Errorf - // Trace is a shortcut for github.com/pingcap/errors.Trace. - Trace = perrors.Trace - // Cause is a shortcut for github.com/pingcap/errors.Cause. - Cause = perrors.Cause - // Annotate is a shortcut for github.com/pingcap/errors.Annotate. - Annotate = perrors.Annotate - // Annotatef is a shortcut for github.com/pingcap/errors.Annotatef. - Annotatef = perrors.Annotatef - // WithMessage is a shortcut for github.com/pingcap/errors.WithMessage. - WithMessage = perrors.WithMessage + Is = errors.Is + As = errors.As + New = perrors.New + Errorf = perrors.Errorf + Trace = perrors.Trace + Cause = perrors.Cause + Annotate = perrors.Annotate + Annotatef = perrors.Annotatef + WithMessage = perrors.WithMessage + WithStack = perrors.WithStack + ErrorStack = perrors.ErrorStack + ErrorEqual = perrors.ErrorEqual + NewNoStackError = perrors.NewNoStackError + NotFoundf = perrors.NotFoundf + NotValidf = perrors.NotValidf + Normalize = perrors.Normalize + RFCCodeText = perrors.RFCCodeText + Unwrap = perrors.Unwrap + RedactLogEnabled = &perrors.RedactLogEnabled ) diff --git a/pkg/errors/utils.go b/pkg/errors/utils.go index 83d853f2a8..1db9c8c4bd 100644 --- a/pkg/errors/utils.go +++ b/pkg/errors/utils.go @@ -17,7 +17,6 @@ import ( "strings" gmysql "github.com/go-sql-driver/mysql" - "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/errno" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser/mysql" @@ -28,13 +27,14 @@ import ( // IsTableNotExistsErr is used to check if the error is a table not exists error. func IsTableNotExistsErr(err error) bool { - err = errors.Cause(err) - mysqlErr, ok := err.(*gmysql.MySQLError) + err = Cause(err) + var mysqlErr *gmysql.MySQLError + ok := As(err, &mysqlErr) if !ok { return false } - errCode := errors.ErrCode(mysqlErr.Number) + errCode := ErrCode(mysqlErr.Number) return errCode == infoschema.ErrTableNotExists.Code() || errCode == infoschema.ErrDatabaseNotExists.Code() } @@ -44,13 +44,14 @@ func IsTableNotExistsErr(err error) bool { // DDL's error definition: https://github.com/pingcap/tidb/blob/master/ddl/ddl.go // tidb/mysql error code definition: https://github.com/pingcap/tidb/blob/master/mysql/errcode.go func IsIgnorableMySQLDDLError(err error) bool { - err = errors.Cause(err) - mysqlErr, ok := err.(*gmysql.MySQLError) + err = Cause(err) + var mysqlErr *gmysql.MySQLError + ok := As(err, &mysqlErr) if !ok { return false } - errCode := errors.ErrCode(mysqlErr.Number) + errCode := ErrCode(mysqlErr.Number) switch errCode { case infoschema.ErrDatabaseExists.Code(), infoschema.ErrDatabaseDropExists.Code(), infoschema.ErrTableExists.Code(), infoschema.ErrTableDropExists.Code(), @@ -93,8 +94,9 @@ func IsRetryableDDLError(err error) bool { return true } - err = errors.Cause(err) - mysqlErr, ok := err.(*gmysql.MySQLError) + err = Cause(err) + var mysqlErr *gmysql.MySQLError + ok := As(err, &mysqlErr) if !ok { return false } @@ -121,8 +123,9 @@ func IsRetryableDDLError(err error) bool { // IsAccessDeniedError checks if the error is an access denied error. func IsAccessDeniedError(err error) bool { - err = errors.Cause(err) - mysqlErr, ok := err.(*gmysql.MySQLError) + err = Cause(err) + var mysqlErr *gmysql.MySQLError + ok := As(err, &mysqlErr) if !ok { return false } @@ -132,8 +135,9 @@ func IsAccessDeniedError(err error) bool { // IsSyncPointIgnoreError returns whether the error is ignorable for syncpoint. func IsSyncPointIgnoreError(err error) bool { - err = errors.Cause(err) - mysqlErr, ok := err.(*gmysql.MySQLError) + err = Cause(err) + var mysqlErr *gmysql.MySQLError + ok := As(err, &mysqlErr) if !ok { return false } @@ -147,22 +151,21 @@ func IsRetryableEtcdError(err error) bool { if err == nil { return false } - etcdErr := errors.Cause(err) + etcdErr := Cause(err) - switch etcdErr { - // Etcd ResourceExhausted errors, may recover after some time - case v3rpc.ErrNoSpace, v3rpc.ErrTooManyRequests: + if Is(etcdErr, v3rpc.ErrNoSpace) || Is(etcdErr, v3rpc.ErrTooManyRequests) { return true + } // Etcd Unavailable errors, may be available after some time // https://github.com/etcd-io/etcd/pull/9934/files#diff-6d8785d0c9eaf96bc3e2b29c36493c04R162-R167 // ErrStopped: // one of the etcd nodes stopped from failure injection // ErrNotCapable: // capability check has not been done (in the beginning) - case v3rpc.ErrNoLeader, v3rpc.ErrLeaderChanged, v3rpc.ErrNotCapable, v3rpc.ErrStopped, v3rpc.ErrTimeout, - v3rpc.ErrTimeoutDueToLeaderFail, v3rpc.ErrGRPCTimeoutDueToConnectionLost, v3rpc.ErrUnhealthy: + if Is(etcdErr, v3rpc.ErrNoLeader) || Is(etcdErr, v3rpc.ErrLeaderChanged) || Is(etcdErr, v3rpc.ErrNotCapable) || + Is(etcdErr, v3rpc.ErrStopped) || Is(etcdErr, v3rpc.ErrTimeout) || Is(etcdErr, v3rpc.ErrTimeoutDueToLeaderFail) || + Is(etcdErr, v3rpc.ErrGRPCTimeoutDueToConnectionLost) || Is(etcdErr, v3rpc.ErrUnhealthy) { return true - default: } // when the PD instance was deleted from the PD cluster, it may meet different errors. // retry on such error make cdc robust to PD / ETCD cluster member removal. diff --git a/pkg/etcd/client.go b/pkg/etcd/client.go index cf9368f1d2..c8fffb7afe 100644 --- a/pkg/etcd/client.go +++ b/pkg/etcd/client.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/ticdc/pkg/util" "github.com/prometheus/client_golang/prometheus" "github.com/tikv/pd/pkg/errs" - "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" "go.etcd.io/etcd/client/pkg/v3/logutil" clientV3 "go.etcd.io/etcd/client/v3" @@ -614,5 +613,5 @@ func IsHealthy(ctx context.Context, client *clientv3.Client) bool { _, err := client.Get(ctx, healthyPath) // permission denied is OK since proposal goes through consensus to get it // See: https://github.com/etcd-io/etcd/blob/85b640cee793e25f3837c47200089d14a8392dc7/etcdctl/ctlv3/command/ep_command.go#L124 - return err == nil || err == rpctypes.ErrPermissionDenied + return err == nil || cerror.Is(err, v3rpc.ErrPermissionDenied) } diff --git a/pkg/eventservice/dispatcher_stat.go b/pkg/eventservice/dispatcher_stat.go index 552a684ea2..2f4319b14a 100644 --- a/pkg/eventservice/dispatcher_stat.go +++ b/pkg/eventservice/dispatcher_stat.go @@ -327,14 +327,6 @@ func (w *wrapEvent) reset() { wrapEventPool.Put(w) } -func (w *wrapEvent) getDispatcherID() common.DispatcherID { - e, ok := w.e.(pevent.Event) - if !ok { - log.Panic("cast event failed", zap.Any("event", w.e)) - } - return e.GetDispatcherID() -} - func newWrapHandshakeEvent(serverID node.ID, e pevent.HandshakeEvent) *wrapEvent { w := getWrapEvent() w.serverID = serverID diff --git a/pkg/eventservice/event_scanner.go b/pkg/eventservice/event_scanner.go index 7b8c712e27..b082ccfc13 100644 --- a/pkg/eventservice/event_scanner.go +++ b/pkg/eventservice/event_scanner.go @@ -15,7 +15,6 @@ package eventservice import ( "context" - "errors" "time" "github.com/pingcap/log" @@ -24,6 +23,7 @@ import ( "github.com/pingcap/ticdc/pkg/common" "github.com/pingcap/ticdc/pkg/common/event" "github.com/pingcap/ticdc/pkg/config" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/filter" "github.com/pingcap/ticdc/pkg/integrity" "github.com/pingcap/ticdc/pkg/metrics" @@ -490,7 +490,7 @@ func (s *session) appendEvents(events []event.Event) { batchDML := item.(*event.BatchDMLEvent) s.eventBytes += int64(len(batchDML.DMLEvents)) } else { - s.eventBytes += 1 + s.eventBytes++ } } } diff --git a/pkg/filter/expr_filter.go b/pkg/filter/expr_filter.go index 3e2f0fb5a9..652cfcdc71 100644 --- a/pkg/filter/expr_filter.go +++ b/pkg/filter/expr_filter.go @@ -17,11 +17,10 @@ import ( "strings" "sync" - "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/ticdc/pkg/common" "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/util" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/meta/model" @@ -62,7 +61,7 @@ func newExprFilterRule( ) (*dmlExprFilterRule, error) { tf, err := tfilter.Parse(cfg.Matcher) if err != nil { - return nil, cerror.WrapError(cerror.ErrFilterRuleInvalid, err, cfg.Matcher) + return nil, errors.WrapError(errors.ErrFilterRuleInvalid, err, cfg.Matcher) } ret := &dmlExprFilterRule{ @@ -88,25 +87,25 @@ func newExprFilterRule( // _, _, err := p.ParseSQL(completeExpression(r.config.IgnoreInsertValueExpr)) // if err != nil { // log.Error("failed to parse expression", zap.Error(err)) -// return cerror.ErrExpressionParseFailed. +// return errors.ErrExpressionParseFailed. // FastGenByArgs(r.config.IgnoreInsertValueExpr) // } // _, _, err = p.ParseSQL(completeExpression(r.config.IgnoreUpdateNewValueExpr)) // if err != nil { // log.Error("failed to parse expression", zap.Error(err)) -// return cerror.ErrExpressionParseFailed. +// return errors.ErrExpressionParseFailed. // FastGenByArgs(r.config.IgnoreUpdateNewValueExpr) // } // _, _, err = p.ParseSQL(completeExpression(r.config.IgnoreUpdateOldValueExpr)) // if err != nil { // log.Error("failed to parse expression", zap.Error(err)) -// return cerror.ErrExpressionParseFailed. +// return errors.ErrExpressionParseFailed. // FastGenByArgs(r.config.IgnoreUpdateOldValueExpr) // } // _, _, err = p.ParseSQL(completeExpression(r.config.IgnoreDeleteValueExpr)) // if err != nil { // log.Error("failed to parse expression", zap.Error(err)) -// return cerror.ErrExpressionParseFailed. +// return errors.ErrExpressionParseFailed. // FastGenByArgs(r.config.IgnoreDeleteValueExpr) // } // verify expression filter rule. @@ -240,11 +239,11 @@ func (r *dmlExprFilterRule) getSimpleExprOfTable( log.Error("meet unknown column when generating expression", zap.String("expression", expr), zap.Error(err)) - return nil, cerror.ErrExpressionColumnNotFound. + return nil, errors.ErrExpressionColumnNotFound. FastGenByArgs(getColumnFromError(err), tableInfo.Name.String(), expr) } log.Error("failed to parse expression", zap.Error(err)) - return nil, cerror.ErrExpressionParseFailed.FastGenByArgs(err, expr) + return nil, errors.ErrExpressionParseFailed.FastGenByArgs(err, expr) } return e, nil } @@ -340,8 +339,6 @@ func (r *dmlExprFilterRule) buildRowWithVirtualColumns( } vColOffsets, vColFts := collectVirtualColumnOffsetsAndTypes(r.sessCtx.GetExprCtx().GetEvalCtx(), columns) - err = table.FillVirtualColumnValue(vColFts, vColOffsets, columns, tableInfo.GetColumns(), r.sessCtx.GetExprCtx(), row.Chunk()) - err = table.FillVirtualColumnValue(vColFts, vColOffsets, columns, tableInfo.GetColumns(), r.sessCtx.GetExprCtx(), row.Chunk()) if err != nil { return chunk.Row{}, err @@ -468,10 +465,10 @@ func (f *dmlExprFilter) shouldSkipDML( for _, rule := range rules { ignore, err := rule.shouldSkipDML(dmlType, preRow, row, tableInfo) if err != nil { - if cerror.ShouldFailChangefeed(err) { + if errors.ShouldFailChangefeed(err) { return false, err } - return false, cerror.WrapError(cerror.ErrFailedToFilterDML, err, row) + return false, errors.WrapError(errors.ErrFailedToFilterDML, err, row) } if ignore { return true, nil diff --git a/pkg/filter/utils.go b/pkg/filter/utils.go index f75562e56e..d02e256717 100644 --- a/pkg/filter/utils.go +++ b/pkg/filter/utils.go @@ -14,12 +14,11 @@ package filter import ( - "fmt" "strings" bf "github.com/pingcap/ticdc/pkg/binlog-filter" "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/errors" timodel "github.com/pingcap/tidb/pkg/meta/model" tifilter "github.com/pingcap/tidb/pkg/util/filter" tfilter "github.com/pingcap/tidb/pkg/util/table-filter" @@ -57,7 +56,7 @@ func VerifyTableRules(cfg *config.FilterConfig) (tfilter.Filter, error) { } f, err := tfilter.Parse(rules) if err != nil { - return nil, cerror.WrapError(cerror.ErrFilterRuleInvalid, err, cfg) + return nil, errors.WrapError(errors.ErrFilterRuleInvalid, err, cfg) } return f, nil @@ -149,10 +148,3 @@ func SupportedEventTypes() []bf.EventType { } return supportedEventTypes } - -func completeExpression(suffix string) string { - if suffix == "" { - return suffix - } - return fmt.Sprintf("select * from t where %s", suffix) -} diff --git a/pkg/httputil/httputil.go b/pkg/httputil/httputil.go index b89e74fef9..e7f57d5a62 100644 --- a/pkg/httputil/httputil.go +++ b/pkg/httputil/httputil.go @@ -21,7 +21,7 @@ import ( "strings" "time" - "github.com/pingcap/errors" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/security" ) @@ -111,7 +111,9 @@ func (c *Client) DoRequest( if err != nil { return nil, errors.Trace(err) } - defer resp.Body.Close() + defer func() { + _ = resp.Body.Close() + }() content, err := io.ReadAll(resp.Body) if err != nil { diff --git a/pkg/logger/log.go b/pkg/logger/log.go index 84fa2c49d9..786ecb982c 100644 --- a/pkg/logger/log.go +++ b/pkg/logger/log.go @@ -24,8 +24,8 @@ import ( "github.com/IBM/sarama" "github.com/gin-gonic/gin" "github.com/go-sql-driver/mysql" - "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/pingcap/ticdc/pkg/errors" "go.uber.org/zap" "go.uber.org/zap/zapcore" "google.golang.org/grpc/grpclog" @@ -232,7 +232,7 @@ func initOptionalComponent(op *loggerOp, cfg *Config) error { func ZapErrorFilter(err error, filterErrors ...error) zap.Field { cause := errors.Cause(err) for _, ferr := range filterErrors { - if cause == ferr { + if errors.Is(cause, ferr) { return zap.Error(nil) } } @@ -334,7 +334,7 @@ func ErrorFilterContextCanceled(logger *zap.Logger, msg string, fields ...zap.Fi } case zapcore.ErrorType: err, ok := field.Interface.(error) - if ok && errors.Cause(err) == context.Canceled { + if ok && errors.Is(errors.Cause(err), context.Canceled) { return } } diff --git a/pkg/messaging/message_center.go b/pkg/messaging/message_center.go index faa94f555d..0a11c31dab 100644 --- a/pkg/messaging/message_center.go +++ b/pkg/messaging/message_center.go @@ -20,10 +20,9 @@ import ( "sync" "time" - "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/ticdc/pkg/config" - pkgerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/messaging/proto" "github.com/pingcap/ticdc/pkg/metrics" "github.com/pingcap/ticdc/pkg/node" @@ -298,7 +297,7 @@ func (mc *messageCenter) SendEvent(msg *TargetMessage) error { // For example, if the target is not discovered yet, the caller can retry later. // If the target is removed, the caller must remove the objects that was sending // message to this target to avoid blocking. - return pkgerror.AppError{Type: pkgerror.ErrorTypeTargetNotFound, Reason: fmt.Sprintf("Target %s not found", msg.To)} + return errors.AppError{Type: errors.ErrorTypeTargetNotFound, Reason: fmt.Sprintf("Target %s not found", msg.To)} } return target.sendEvent(msg) } @@ -316,7 +315,7 @@ func (mc *messageCenter) SendCommand(msg *TargetMessage) error { target, ok := mc.remoteTargets.m[msg.To] mc.remoteTargets.RUnlock() if !ok { - return errors.WithStack(pkgerror.AppError{Type: pkgerror.ErrorTypeTargetNotFound, Reason: fmt.Sprintf("Target %v not found", msg.To.String())}) + return errors.WithStack(errors.AppError{Type: errors.ErrorTypeTargetNotFound, Reason: fmt.Sprintf("Target %v not found", msg.To.String())}) } return target.sendCommand(msg) } @@ -446,7 +445,7 @@ func (s *grpcServer) handleConnect(stream proto.MessageService_StreamMessagesSer to := node.ID(msg.To) if to != s.id() { - err := pkgerror.AppError{Type: pkgerror.ErrorTypeTargetMismatch, Reason: fmt.Sprintf("The receiver %s not match with the message center id %s", to, s.id())} + err := errors.AppError{Type: errors.ErrorTypeTargetMismatch, Reason: fmt.Sprintf("The receiver %s not match with the message center id %s", to, s.id())} log.Error("Target mismatch", zap.Error(err)) return err } @@ -468,12 +467,12 @@ func (s *grpcServer) handleConnect(stream proto.MessageService_StreamMessagesSer zap.Stringer("localID", s.messageCenter.id), zap.String("localAddr", s.messageCenter.addr), zap.Stringer("remoteID", targetId)) - err := &pkgerror.AppError{Type: pkgerror.ErrorTypeTargetNotFound, Reason: fmt.Sprintf("Target %s not found", targetId)} + err := &errors.AppError{Type: errors.ErrorTypeTargetNotFound, Reason: fmt.Sprintf("Target %s not found", targetId)} return err } return nil }, retry.WithIsRetryableErr(func(err error) bool { - return err != context.DeadlineExceeded + return !errors.Is(err, context.DeadlineExceeded) }), retry.WithMaxTries(10)) if err != nil { log.Error("Failed to get remote target", zap.Error(err)) diff --git a/pkg/messaging/remote_target.go b/pkg/messaging/remote_target.go index d7e09a3113..852ebe6336 100644 --- a/pkg/messaging/remote_target.go +++ b/pkg/messaging/remote_target.go @@ -19,11 +19,10 @@ import ( "sync" "time" - "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/pingcap/ticdc/pkg/config" - . "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/messaging/proto" "github.com/pingcap/ticdc/pkg/metrics" "github.com/pingcap/ticdc/pkg/node" @@ -39,12 +38,6 @@ const ( reconnectInterval = 2 * time.Second streamTypeEvent = "event" streamTypeCommand = "command" - - eventRecvCh = "eventRecvCh" - commandRecvCh = "commandRecvCh" - - eventSendCh = "eventSendCh" - commandSendCh = "commandSendCh" ) type streamSession struct { @@ -117,7 +110,7 @@ func (s *remoteMessageTarget) isReadyToSend() bool { func (s *remoteMessageTarget) sendEvent(msg ...*TargetMessage) error { if !s.isReadyToSend() { s.errorCounter.Inc() - return AppError{Type: ErrorTypeConnectionNotFound, Reason: genSendErrorMsg("Stream not ready", string(s.messageCenterID), s.localAddr, string(s.targetId), s.targetAddr)} + return errors.AppError{Type: errors.ErrorTypeConnectionNotFound, Reason: genSendErrorMsg("Stream not ready", string(s.messageCenterID), s.localAddr, string(s.targetId), s.targetAddr)} } // Create message with EVENT type @@ -126,13 +119,13 @@ func (s *remoteMessageTarget) sendEvent(msg ...*TargetMessage) error { select { case <-s.ctx.Done(): s.errorCounter.Inc() - return AppError{Type: ErrorTypeConnectionNotFound, Reason: genSendErrorMsg("Stream has been closed", string(s.messageCenterID), s.localAddr, string(s.targetId), s.targetAddr)} + return errors.AppError{Type: errors.ErrorTypeConnectionNotFound, Reason: genSendErrorMsg("Stream has been closed", string(s.messageCenterID), s.localAddr, string(s.targetId), s.targetAddr)} case s.sendEventCh <- protoMsg: s.sendEventCounter.Add(float64(len(msg))) return nil default: s.congestedEventErrorCounter.Inc() - return AppError{Type: ErrorTypeMessageCongested, Reason: genSendErrorMsg("Send event message is congested", string(s.messageCenterID), s.localAddr, string(s.targetId), s.targetAddr)} + return errors.AppError{Type: errors.ErrorTypeMessageCongested, Reason: genSendErrorMsg("Send event message is congested", string(s.messageCenterID), s.localAddr, string(s.targetId), s.targetAddr)} } } @@ -140,7 +133,7 @@ func (s *remoteMessageTarget) sendEvent(msg ...*TargetMessage) error { func (s *remoteMessageTarget) sendCommand(msg ...*TargetMessage) error { if !s.isReadyToSend() { s.errorCounter.Inc() - return AppError{Type: ErrorTypeConnectionNotFound, Reason: genSendErrorMsg("Stream not ready", string(s.messageCenterID), s.localAddr, string(s.targetId), s.targetAddr)} + return errors.AppError{Type: errors.ErrorTypeConnectionNotFound, Reason: genSendErrorMsg("Stream not ready", string(s.messageCenterID), s.localAddr, string(s.targetId), s.targetAddr)} } // Create message with COMMAND type @@ -149,13 +142,13 @@ func (s *remoteMessageTarget) sendCommand(msg ...*TargetMessage) error { select { case <-s.ctx.Done(): s.errorCounter.Inc() - return AppError{Type: ErrorTypeConnectionNotFound, Reason: genSendErrorMsg("Stream has been closed", string(s.messageCenterID), s.localAddr, string(s.targetId), s.targetAddr)} + return errors.AppError{Type: errors.ErrorTypeConnectionNotFound, Reason: genSendErrorMsg("Stream has been closed", string(s.messageCenterID), s.localAddr, string(s.targetId), s.targetAddr)} case s.sendCmdCh <- protoMsg: s.sendCmdCounter.Add(float64(len(msg))) return nil default: s.congestedCmdErrorCounter.Inc() - return AppError{Type: ErrorTypeMessageCongested, Reason: genSendErrorMsg("Send command message is congested", string(s.messageCenterID), s.localAddr, string(s.targetId), s.targetAddr)} + return errors.AppError{Type: errors.ErrorTypeMessageCongested, Reason: genSendErrorMsg("Send command message is congested", string(s.messageCenterID), s.localAddr, string(s.targetId), s.targetAddr)} } } @@ -280,8 +273,8 @@ func (s *remoteMessageTarget) connect() error { zap.Any("remoteID", s.targetId), zap.Error(err)) - return AppError{ - Type: ErrorTypeConnectionFailed, + return errors.AppError{ + Type: errors.ErrorTypeConnectionFailed, Reason: fmt.Sprintf("Cannot create grpc client on address %s, error: %s", s.targetAddr, err.Error()), } } @@ -312,8 +305,8 @@ func (s *remoteMessageTarget) connect() error { zap.String("remoteAddr", s.targetAddr), zap.Error(err)) - err = AppError{ - Type: ErrorTypeConnectionFailed, + err = errors.AppError{ + Type: errors.ErrorTypeConnectionFailed, Reason: fmt.Sprintf("Cannot open bidirectional grpc stream, error: %s", errors.Trace(err).Error()), } outerErr = err @@ -334,7 +327,7 @@ func (s *remoteMessageTarget) connect() error { hsBytes, err := handshake.Marshal() if err != nil { log.Error("Failed to marshal handshake message", zap.Error(err)) - err = AppError{Type: ErrorTypeMessageSendFailed, Reason: errors.Trace(err).Error()} + err = errors.AppError{Type: errors.ErrorTypeMessageSendFailed, Reason: errors.Trace(err).Error()} outerErr = err return false } @@ -354,8 +347,8 @@ func (s *remoteMessageTarget) connect() error { zap.Any("remoteID", s.targetId), zap.String("remoteAddr", s.targetAddr), zap.Error(err)) - err = AppError{ - Type: ErrorTypeMessageSendFailed, + err = errors.AppError{ + Type: errors.ErrorTypeMessageSendFailed, Reason: fmt.Sprintf("Failed to send handshake, error: %s", errors.Trace(err).Error()), } outerErr = err @@ -547,7 +540,7 @@ func (s *remoteMessageTarget) runSendMessages(ctx context.Context, streamType st zap.String("remoteAddr", s.targetAddr), zap.String("streamType", streamType), zap.Stringer("message", msg)) - err = AppError{Type: ErrorTypeMessageSendFailed, Reason: errors.Trace(err).Error()} + err = errors.AppError{Type: errors.ErrorTypeMessageSendFailed, Reason: errors.Trace(err).Error()} return err } } @@ -622,7 +615,7 @@ func (s *remoteMessageTarget) handleIncomingMessage(ctx context.Context, stream zap.String("localAddr", s.localAddr), zap.Stringer("remoteID", s.targetId), zap.String("remoteAddr", s.targetAddr)) - err = AppError{Type: ErrorTypeMessageReceiveFailed, Reason: errors.Trace(err).Error()} + err = errors.AppError{Type: errors.ErrorTypeMessageReceiveFailed, Reason: errors.Trace(err).Error()} return err } diff --git a/pkg/messaging/stream.go b/pkg/messaging/stream.go index 17c3771d78..5f99a4fb66 100644 --- a/pkg/messaging/stream.go +++ b/pkg/messaging/stream.go @@ -14,8 +14,6 @@ package messaging import ( - "sync/atomic" - "github.com/pingcap/ticdc/pkg/messaging/proto" ) @@ -26,41 +24,3 @@ type grpcStream interface { Send(*proto.Message) error Recv() (*proto.Message, error) } - -var streamGenerator atomic.Uint64 - -type streamWrapper struct { - grpcStream - id uint64 - streamType string -} - -// newStreamWrapper creates a new stream wrapper. -func newStreamWrapper(stream grpcStream, streamType string) *streamWrapper { - return &streamWrapper{ - grpcStream: stream, - id: streamGenerator.Add(1), - streamType: streamType, - } -} - -func (s *streamWrapper) Send(msg *proto.Message) error { - return s.grpcStream.Send(msg) -} - -func (s *streamWrapper) Recv() (*proto.Message, error) { - return s.grpcStream.Recv() -} - -func (s *streamWrapper) ID() uint64 { - return s.id -} - -func (s *streamWrapper) StreamType() string { - return s.streamType -} - -// Equals returns true if the two streams are the same. -func (s *streamWrapper) Equals(other *streamWrapper) bool { - return s.id == other.id -} diff --git a/pkg/orchestrator/etcd_worker_test.go b/pkg/orchestrator/etcd_worker_test.go index ca8303d4d9..c3e38ff06d 100644 --- a/pkg/orchestrator/etcd_worker_test.go +++ b/pkg/orchestrator/etcd_worker_test.go @@ -23,9 +23,8 @@ import ( "testing" "time" - "github.com/pingcap/errors" "github.com/pingcap/log" - cerrors "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/etcd" "github.com/pingcap/ticdc/pkg/migrate" "github.com/pingcap/ticdc/pkg/orchestrator/util" @@ -52,7 +51,7 @@ type simpleReactor struct { func (s *simpleReactor) Tick(_ context.Context, state ReactorState) (nextState ReactorState, err error) { if s.tickCount >= totalTicksPerReactor { - return s.state, cerrors.ErrReactorFinished + return s.state, errors.ErrReactorFinished.FastGenByArgs() } s.tickCount++ @@ -272,8 +271,8 @@ func TestEtcdSum(t *testing.T) { } err = errg.Wait() - if err != nil && (errors.Cause(err) == context.DeadlineExceeded || - errors.Cause(err) == context.Canceled || + if err != nil && (errors.Is(err, context.DeadlineExceeded) || + errors.Is(err, context.Canceled) || strings.Contains(err.Error(), "etcdserver: request timeout")) { return } @@ -323,7 +322,7 @@ func (r *linearizabilityReactor) Tick(ctx context.Context, state ReactorState) ( r.tickCount++ } if r.state.val == 1999 { - return r.state, cerrors.ErrReactorFinished + return r.state, errors.ErrReactorFinished.FastGenByArgs() } r.state.isUpdated = false return r.state, nil @@ -423,7 +422,7 @@ func (r *finishedReactor) Tick(ctx context.Context, state ReactorState) (nextSta r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, changed bool, err error) { return nil, true, nil }) - return r.state, cerrors.ErrReactorFinished + return r.state, errors.ErrReactorFinished.FastGenByArgs() } func TestFinished(t *testing.T) { @@ -493,7 +492,7 @@ func (r *coverReactor) Tick(ctx context.Context, state ReactorState) (nextState r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, changed bool, err error) { return append(old, []byte("fin")...), true, nil }) - return r.state, cerrors.ErrReactorFinished + return r.state, errors.ErrReactorFinished.FastGenByArgs() } func TestCover(t *testing.T) { @@ -572,7 +571,7 @@ func (r *emptyTxnReactor) Tick(ctx context.Context, state ReactorState) (nextSta r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, changed bool, err error) { return []byte("123"), true, nil }) - return r.state, cerrors.ErrReactorFinished + return r.state, errors.ErrReactorFinished.FastGenByArgs() } func TestEmptyTxn(t *testing.T) { @@ -641,7 +640,7 @@ func (r *emptyOrNilReactor) Tick(ctx context.Context, state ReactorState) (nextS r.state.AppendPatch(util.NewEtcdKey(r.prefix+"/key2"), func(old []byte) (newValue []byte, changed bool, err error) { return nil, true, nil }) - return r.state, cerrors.ErrReactorFinished + return r.state, errors.ErrReactorFinished.FastGenByArgs() } func TestEmptyOrNil(t *testing.T) { @@ -689,7 +688,7 @@ func (r *modifyOneReactor) Tick(ctx context.Context, state ReactorState) (nextSt if !r.finished { r.finished = true } else { - return r.state, cerrors.ErrReactorFinished.GenWithStackByArgs() + return r.state, errors.ErrReactorFinished.GenWithStackByArgs() } if r.waitOnCh != nil { select { @@ -777,8 +776,8 @@ func TestModifyAfterDelete(t *testing.T) { } func TestRetryableError(t *testing.T) { - require.True(t, isRetryableError(cerrors.ErrEtcdTryAgain)) - require.True(t, isRetryableError(cerrors.ErrReachMaxTry.Wrap(rpctypes.ErrTimeoutDueToLeaderFail))) + require.True(t, isRetryableError(errors.ErrEtcdTryAgain)) + require.True(t, isRetryableError(errors.ErrReachMaxTry.Wrap(rpctypes.ErrTimeoutDueToLeaderFail))) require.True(t, isRetryableError(errors.Trace(context.DeadlineExceeded))) require.False(t, isRetryableError(context.Canceled)) } diff --git a/pkg/pdutil/api_client.go b/pkg/pdutil/api_client.go index 3abbd793ad..ed70db9ee3 100644 --- a/pkg/pdutil/api_client.go +++ b/pkg/pdutil/api_client.go @@ -27,13 +27,12 @@ import ( "strings" "time" - "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/log" "github.com/pingcap/ticdc/heartbeatpb" "github.com/pingcap/ticdc/pkg/common" "github.com/pingcap/ticdc/pkg/config/kerneltype" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/httputil" "github.com/pingcap/ticdc/pkg/retry" "github.com/pingcap/ticdc/pkg/security" @@ -187,11 +186,7 @@ func (pc *pdAPIClient) UpdateMetaLabel(ctx context.Context) error { retry.WithBackoffBaseDelay(200), retry.WithBackoffMaxDelay(4000), retry.WithIsRetryableErr(func(err error) bool { - switch errors.Cause(err) { - case context.Canceled: - return false - } - return true + return !errors.Is(errors.Cause(err), context.Canceled) })) return err } @@ -283,8 +278,7 @@ func (pc *pdAPIClient) scanRegions( // Because start key is less than end key, there must be some regions. log.Error("fail to scan region, missing region", zap.String("endpoint", endpoint)) - return nil, cerror.WrapError(cerror.ErrInternalServerError, - fmt.Errorf("fail to scan region, missing region")) + return nil, errors.ErrInternalServerError.GenWithStack("fail to scan region, missing region") } if r[0].StartKey != startKeyHex { r[0].StartKey = strings.ToUpper(hex.EncodeToString(startKey)) @@ -364,11 +358,7 @@ func (pc *pdAPIClient) ListGcServiceSafePoint( } return nil }, retry.WithMaxTries(defaultMaxRetry), retry.WithIsRetryableErr(func(err error) bool { - switch errors.Cause(err) { - case context.Canceled: - return false - } - return true + return !errors.Is(errors.Cause(err), context.Canceled) })) return resp, err } diff --git a/pkg/redo/reader/file.go b/pkg/redo/reader/file.go index ef5da20027..73b1f29c27 100644 --- a/pkg/redo/reader/file.go +++ b/pkg/redo/reader/file.go @@ -29,11 +29,10 @@ import ( "sync" "time" - "github.com/pingcap/errors" "github.com/pingcap/log" pevent "github.com/pingcap/ticdc/pkg/common/event" "github.com/pingcap/ticdc/pkg/compression" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/redo" "github.com/pingcap/ticdc/pkg/redo/codec" "github.com/pingcap/ticdc/pkg/redo/writer" @@ -85,7 +84,7 @@ type reader struct { func newReaders(ctx context.Context, cfg *readerConfig) ([]fileReader, error) { if cfg == nil { - return nil, cerror.WrapError(cerror.ErrRedoConfigInvalid, errors.New("readerConfig can not be nil")) + return nil, errors.ErrRedoConfigInvalid.GenWithStack("reader Config can not be nil") } if !cfg.useExternalStorage { log.Panic("external storage is not enabled, please check your configuration") @@ -122,7 +121,7 @@ func downLoadAndSortFiles(ctx context.Context, cfg *readerConfig) ([]io.ReadClos // create temp dir in local storage err := os.MkdirAll(dir, redo.DefaultDirMode) if err != nil { - return nil, cerror.WrapError(cerror.ErrRedoFileOp, err) + return nil, errors.WrapError(errors.ErrRedoFileOp, err) } // get all files @@ -169,7 +168,7 @@ func downLoadAndSortFiles(ctx context.Context, cfg *readerConfig) ([]io.ReadClos if os.IsNotExist(err) { continue } - return nil, cerror.WrapError(cerror.ErrRedoFileOp, err) + return nil, errors.WrapError(errors.ErrRedoFileOp, err) } ret = append(ret, f) } @@ -202,7 +201,7 @@ func selectDownLoadFile( return nil }) if err != nil { - return nil, cerror.WrapError(cerror.ErrExternalStorageAPI, err) + return nil, errors.WrapError(errors.ErrExternalStorageAPI, err) } return files, nil @@ -219,13 +218,15 @@ func readAllFromBuffer(buf []byte) (logHeap, error) { r := &reader{ br: bytes.NewReader(buf), } - defer r.Close() + defer func() { + _ = r.Close() + }() h := logHeap{} for { rl, err := r.Read() if err != nil { - if err != io.EOF { + if !errors.Is(err, io.EOF) { return nil, err } break @@ -253,7 +254,7 @@ func sortAndWriteFile( fileContent, err := extStorage.ReadFile(egCtx, fileName) if err != nil { - return cerror.WrapError(cerror.ErrExternalStorageAPI, err) + return errors.WrapError(errors.ErrExternalStorageAPI, err) } if len(fileContent) == 0 { log.Warn("download file is empty", zap.String("file", fileName)) @@ -290,7 +291,7 @@ func sortAndWriteFile( } data, err := codec.MarshalRedoLog(item, nil) if err != nil { - return cerror.WrapError(cerror.ErrMarshalFailed, err) + return errors.WrapError(errors.ErrMarshalFailed, err) } _, err = w.Write(data) if err != nil { @@ -327,23 +328,23 @@ func (r *reader) Read() (*pevent.RedoLog, error) { lenField, err := readInt64(r.br) if err != nil { - if err == io.EOF { + if errors.Is(err, io.EOF) { return nil, err } - return nil, cerror.WrapError(cerror.ErrRedoFileOp, err) + return nil, errors.WrapError(errors.ErrRedoFileOp, err) } recBytes, padBytes := decodeFrameSize(lenField) data := make([]byte, recBytes+padBytes) _, err = io.ReadFull(r.br, data) if err != nil { - if err == io.EOF || err == io.ErrUnexpectedEOF { + if errors.Is(err, io.EOF) || errors.Is(err, io.ErrUnexpectedEOF) { log.Warn("read redo log have unexpected io error", zap.String("fileName", r.fileName), zap.Error(err)) return nil, io.EOF } - return nil, cerror.WrapError(cerror.ErrRedoFileOp, err) + return nil, errors.WrapError(errors.ErrRedoFileOp, err) } redoLog, _, err := codec.UnmarshalRedoLog(data[:recBytes]) @@ -352,7 +353,7 @@ func (r *reader) Read() (*pevent.RedoLog, error) { // just return io.EOF, since if torn write it is the last redoLog entry return nil, io.EOF } - return nil, cerror.WrapError(cerror.ErrUnmarshalFailed, err) + return nil, errors.WrapError(errors.ErrUnmarshalFailed, err) } // point last valid offset to the end of redoLog @@ -420,5 +421,5 @@ func (r *reader) Close() error { return nil } - return cerror.WrapError(cerror.ErrRedoFileOp, r.closer.Close()) + return errors.WrapError(errors.ErrRedoFileOp, r.closer.Close()) } diff --git a/pkg/redo/reader/reader.go b/pkg/redo/reader/reader.go index a687cf6a8f..27933a1a7f 100644 --- a/pkg/redo/reader/reader.go +++ b/pkg/redo/reader/reader.go @@ -223,7 +223,7 @@ func (l *LogReader) runReader(egCtx context.Context, cfg *readerConfig) error { // read next and push again rl, err := fileReaders[item.idx].Read() if err != nil { - if err != io.EOF { + if !errors.Is(err, io.EOF) { return errors.Trace(err) } continue @@ -338,7 +338,7 @@ func newLogHeap(fileReaders []fileReader) (logHeap, error) { for i := 0; i < len(fileReaders); i++ { rl, err := fileReaders[i].Read() if err != nil { - if err != io.EOF { + if !errors.Is(err, io.EOF) { return nil, err } continue diff --git a/pkg/retry/retry_test.go b/pkg/retry/retry_test.go index dfc11ec8f4..f2c43a4238 100644 --- a/pkg/retry/retry_test.go +++ b/pkg/retry/retry_test.go @@ -19,7 +19,7 @@ import ( "testing" "time" - "github.com/pingcap/errors" + "github.com/pingcap/ticdc/pkg/errors" "github.com/stretchr/testify/require" ) @@ -64,11 +64,7 @@ func TestIsRetryable(t *testing.T) { } err := Do(context.Background(), f, WithMaxTries(3), WithIsRetryableErr(func(err error) bool { - switch errors.Cause(err) { - case context.Canceled: - return false - } - return true + return !errors.Is(errors.Cause(err), context.Canceled) })) require.Equal(t, errors.Cause(err), context.Canceled) diff --git a/pkg/sink/cloudstorage/table_definition.go b/pkg/sink/cloudstorage/table_definition.go index 7800c2bcbb..92ed7d4ef0 100644 --- a/pkg/sink/cloudstorage/table_definition.go +++ b/pkg/sink/cloudstorage/table_definition.go @@ -271,7 +271,7 @@ func (t *TableDefinition) ToTableInfo() (*common.TableInfo, error) { tidbTableInfo.PKIsHandle = true } tidbTableInfo.Columns = append(tidbTableInfo.Columns, tiCol) - nextMockID += 1 + nextMockID++ } info := common.NewTableInfo4Decoder(t.Schema, tidbTableInfo) return info, nil diff --git a/pkg/sink/codec/bootstraper.go b/pkg/sink/codec/bootstraper.go index a57da8360f..223af820ce 100644 --- a/pkg/sink/codec/bootstraper.go +++ b/pkg/sink/codec/bootstraper.go @@ -90,7 +90,7 @@ func (b *bootstrapWorker) run(ctx context.Context) error { case <-ctx.Done(): return ctx.Err() case <-sendTicker.C: - b.activeTables.Range(func(key, value interface{}) bool { + b.activeTables.Range(func(_, value interface{}) bool { table := value.(*tableStatistic) err = b.sendBootstrapMsg(ctx, table) return err == nil diff --git a/pkg/sink/codec/canal/canal_json_message.go b/pkg/sink/codec/canal/canal_json_message.go index 1ec447291c..84f08f4be9 100644 --- a/pkg/sink/codec/canal/canal_json_message.go +++ b/pkg/sink/codec/canal/canal_json_message.go @@ -143,7 +143,3 @@ type canalJSONMessageWithTiDBExtension struct { func (c *canalJSONMessageWithTiDBExtension) getCommitTs() uint64 { return c.Extensions.CommitTs } - -func (c *canalJSONMessageWithTiDBExtension) getRowKey() string { - return c.Extensions.RowKey -} diff --git a/pkg/sink/codec/common/helper.go b/pkg/sink/codec/common/helper.go index e92fc9a102..917e227ff4 100644 --- a/pkg/sink/codec/common/helper.go +++ b/pkg/sink/codec/common/helper.go @@ -23,9 +23,9 @@ import ( "unsafe" mysqlDriver "github.com/go-sql-driver/mysql" - "github.com/pingcap/errors" "github.com/pingcap/log" commonEvent "github.com/pingcap/ticdc/pkg/common/event" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/util" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/mysql" @@ -268,7 +268,8 @@ func queryRowChecksumAux( query := fmt.Sprintf("set @@tidb_snapshot=%d", commitTs) _, err := conn.ExecContext(ctx, query) if err != nil { - mysqlErr, ok := errors.Cause(err).(*mysqlDriver.MySQLError) + var mysqlErr *mysqlDriver.MySQLError + ok := errors.As(err, &mysqlErr) if ok { // Error 8055 (HY000): snapshot is older than GC safe point if mysqlErr.Number == 8055 { @@ -324,7 +325,8 @@ func MustSnapshotQuery( query := fmt.Sprintf("set @@tidb_snapshot=%d", commitTs) _, err = conn.ExecContext(ctx, query) if err != nil { - mysqlErr, ok := errors.Cause(err).(*mysqlDriver.MySQLError) + var mysqlErr *mysqlDriver.MySQLError + ok := errors.As(errors.Cause(err), &mysqlErr) if ok { // Error 8055 (HY000): snapshot is older than GC safe point if mysqlErr.Number == 8055 { diff --git a/pkg/sink/codec/csv/csv_decoder.go b/pkg/sink/codec/csv/csv_decoder.go index 05109c687a..df29abacdb 100644 --- a/pkg/sink/codec/csv/csv_decoder.go +++ b/pkg/sink/codec/csv/csv_decoder.go @@ -110,7 +110,7 @@ func (b *decoder) HasNext() (common.MessageType, bool) { err := b.parser.ReadRow() if err != nil { b.closed = true - if errors.Cause(err) == io.EOF { + if errors.Is(errors.Cause(err), io.EOF) { return common.MessageTypeUnknown, false } log.Panic("read csv row failed", zap.Error(err)) @@ -239,7 +239,7 @@ func csvMsg2RowChangedEvent(csvConfig *common.Config, csvMsg *csvMessage, tableI e.RowTypes = append(e.RowTypes, commonType.RowTypeInsert) } e.Rows = chk - e.Length += 1 + e.Length++ return e, nil } diff --git a/pkg/sink/codec/csv/csv_message_test.go b/pkg/sink/codec/csv/csv_message_test.go index 43f163bcfe..ec409f22c0 100644 --- a/pkg/sink/codec/csv/csv_message_test.go +++ b/pkg/sink/codec/csv/csv_message_test.go @@ -95,7 +95,7 @@ var csvTestColumnsGroup = [][]*csvTestColumnTuple{ }, model.ColumnInfo{ ID: 6, - FieldType: *setFlag(types.NewFieldType(mysql.TypeTiny), uint(mysql.UnsignedFlag)), + FieldType: *setFlag(types.NewFieldType(mysql.TypeTiny), mysql.UnsignedFlag), }, uint64(1), config.BinaryEncodingBase64, diff --git a/pkg/sink/codec/debezium/helper.go b/pkg/sink/codec/debezium/helper.go index f23f3333b4..f7c404019f 100644 --- a/pkg/sink/codec/debezium/helper.go +++ b/pkg/sink/codec/debezium/helper.go @@ -183,10 +183,10 @@ func getLen(ft types.FieldType) int { } case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong: if mysql.HasUnsignedFlag(ft.GetFlag()) { - defaultFlen -= 1 + defaultFlen-- } if ft.GetType() == mysql.TypeTiny && mysql.HasZerofillFlag(ft.GetFlag()) { - defaultFlen += 1 + defaultFlen++ } if flen != defaultFlen { return flen @@ -285,7 +285,7 @@ func getBitFromUint64(n int, v uint64) []byte { binary.LittleEndian.PutUint64(buf[:], v) numBytes := n / 8 if n%8 != 0 { - numBytes += 1 + numBytes++ } return buf[:numBytes] } diff --git a/pkg/sink/codec/open/message.go b/pkg/sink/codec/open/message.go index 811be7e937..7ae2b94886 100644 --- a/pkg/sink/codec/open/message.go +++ b/pkg/sink/codec/open/message.go @@ -123,10 +123,6 @@ func isUnique(flag uint64) bool { return flag&uniqueKeyFlag != 0 } -func isMultiKey(flag uint64) bool { - return flag&multipleKeyFlag != 0 -} - func isNullable(flag uint64) bool { return flag&nullableFlag != 0 } diff --git a/pkg/sink/kafka/claimcheck/claim_check.go b/pkg/sink/kafka/claimcheck/claim_check.go index 052785e2fa..62dc901238 100644 --- a/pkg/sink/kafka/claimcheck/claim_check.go +++ b/pkg/sink/kafka/claimcheck/claim_check.go @@ -20,10 +20,10 @@ import ( "time" "github.com/google/uuid" - "github.com/pingcap/errors" "github.com/pingcap/log" commonType "github.com/pingcap/ticdc/pkg/common" "github.com/pingcap/ticdc/pkg/config" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/sink/codec/common" "github.com/pingcap/ticdc/pkg/util" "github.com/pingcap/tidb/br/pkg/storage" @@ -31,10 +31,6 @@ import ( "go.uber.org/zap" ) -const ( - defaultTimeout = 5 * time.Minute -) - // ClaimCheck manage send message to the claim-check external storage. type ClaimCheck struct { storage storage.ExternalStorage diff --git a/pkg/sink/kafka/options_test.go b/pkg/sink/kafka/options_test.go index 36c2f4e669..3139b5b539 100644 --- a/pkg/sink/kafka/options_test.go +++ b/pkg/sink/kafka/options_test.go @@ -24,10 +24,9 @@ import ( "github.com/IBM/sarama" "github.com/aws/aws-sdk-go/aws" - "github.com/pingcap/errors" commonType "github.com/pingcap/ticdc/pkg/common" "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/sink/codec/common" "github.com/stretchr/testify/require" ) @@ -108,7 +107,7 @@ func TestCompleteOptions(t *testing.T) { require.NoError(t, err) options = NewOptions() err = options.Apply(commonType.NewChangefeedID4Test(commonType.DefaultKeyspaceName, "test"), sinkURI, config.GetDefaultReplicaConfig().Sink) - require.True(t, cerror.ErrKafkaInvalidClientID.Equal(err)) + require.True(t, errors.ErrKafkaInvalidClientID.Equal(err)) } func TestSetPartitionNum(t *testing.T) { @@ -124,7 +123,7 @@ func TestSetPartitionNum(t *testing.T) { options.PartitionNum = 3 err = options.setPartitionNum(2) - require.True(t, cerror.ErrKafkaInvalidPartitionNum.Equal(err)) + require.True(t, errors.ErrKafkaInvalidPartitionNum.Equal(err)) } func TestClientID(t *testing.T) { @@ -832,7 +831,8 @@ func TestAdjustOptionsKeepAlive(t *testing.T) { err := adjustOptions(ctx, adminClient, o, adminClient.GetDefaultMockTopicName()) require.Error(t, err) // The error should be a type conversion error. - _, ok := errors.Cause(err).(*strconv.NumError) + var numError *strconv.NumError + ok := errors.As(errors.Cause(err), &numError) require.True(t, ok, "error should be of type strconv.NumError") }) diff --git a/pkg/sink/mysql/config.go b/pkg/sink/mysql/config.go index 2a44d27b2a..83b5897900 100644 --- a/pkg/sink/mysql/config.go +++ b/pkg/sink/mysql/config.go @@ -25,12 +25,11 @@ import ( dmysql "github.com/go-sql-driver/mysql" lru "github.com/hashicorp/golang-lru" - "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/pingcap/ticdc/pkg/common" "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/security" "github.com/pingcap/ticdc/pkg/sink/sqlmodel" "github.com/pingcap/ticdc/pkg/util" @@ -229,12 +228,12 @@ func (c *Config) Apply( ) (err error) { if sinkURI == nil { log.Error("empty SinkURI") - return cerror.ErrMySQLInvalidConfig.GenWithStack("fail to open MySQL sink, empty SinkURI") + return errors.ErrMySQLInvalidConfig.GenWithStack("fail to open MySQL sink, empty SinkURI") } c.sinkURI = sinkURI scheme := strings.ToLower(sinkURI.Scheme) if !config.IsMySQLCompatibleScheme(scheme) { - return cerror.ErrMySQLInvalidConfig.GenWithStack("can't create MySQL sink with unsupported scheme: %s", scheme) + return errors.ErrMySQLInvalidConfig.GenWithStack("can't create MySQL sink with unsupported scheme: %s", scheme) } if cfg != nil { @@ -385,7 +384,9 @@ func NewMysqlConfigAndDB( if cachePrepStmts { cfg.stmtCache, err = lru.NewWithEvict(prepStmtCacheSize, func(key, value interface{}) { stmt := value.(*sql.Stmt) - stmt.Close() + if err := stmt.Close(); err != nil { + log.Warn("failed to close cached prepared statement", zap.Error(err)) + } }) if err != nil { return nil, nil, err @@ -407,12 +408,12 @@ func NewMysqlConfigAndDB( // IsSinkSafeMode returns whether the sink is in safe mode. func IsSinkSafeMode(sinkURI *url.URL, replicaConfig *config.ReplicaConfig) (bool, error) { if sinkURI == nil { - return false, cerror.ErrMySQLInvalidConfig.GenWithStack("fail to open MySQL sink, empty SinkURI") + return false, errors.ErrMySQLInvalidConfig.GenWithStack("fail to open MySQL sink, empty SinkURI") } scheme := strings.ToLower(sinkURI.Scheme) if !config.IsMySQLCompatibleScheme(scheme) { - return false, cerror.ErrMySQLInvalidConfig.GenWithStack("can't create MySQL sink with unsupported scheme: %s", scheme) + return false, errors.ErrMySQLInvalidConfig.GenWithStack("can't create MySQL sink with unsupported scheme: %s", scheme) } query := sinkURI.Query() var safeMode bool @@ -430,10 +431,10 @@ func getWorkerCount(values url.Values, workerCount *int, workerCountSpecified *b c, err := strconv.Atoi(s) if err != nil { - return cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) + return errors.WrapError(errors.ErrMySQLInvalidConfig, err) } if c <= 0 { - return cerror.WrapError(cerror.ErrMySQLInvalidConfig, + return errors.WrapError(errors.ErrMySQLInvalidConfig, fmt.Errorf("invalid worker-count %d, which must be greater than 0", c)) } if c > maxWorkerCount { @@ -455,10 +456,10 @@ func getMaxTxnRow(values url.Values, maxTxnRow *int) error { c, err := strconv.Atoi(s) if err != nil { - return cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) + return errors.WrapError(errors.ErrMySQLInvalidConfig, err) } if c <= 0 { - return cerror.WrapError(cerror.ErrMySQLInvalidConfig, + return errors.WrapError(errors.ErrMySQLInvalidConfig, fmt.Errorf("invalid max-txn-row %d, which must be greater than 0", c)) } if c > maxMaxTxnRow { @@ -478,10 +479,10 @@ func getMaxMultiUpdateRowCount(values url.Values, maxMultiUpdateRow *int) error c, err := strconv.Atoi(s) if err != nil { - return cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) + return errors.WrapError(errors.ErrMySQLInvalidConfig, err) } if c <= 0 { - return cerror.WrapError(cerror.ErrMySQLInvalidConfig, + return errors.WrapError(errors.ErrMySQLInvalidConfig, fmt.Errorf("invalid max-multi-update-row %d, which must be greater than 0", c)) } if c > maxMaxMultiUpdateRowCount { @@ -501,10 +502,10 @@ func getMaxMultiUpdateRowSize(values url.Values, maxMultiUpdateRowSize *int) err c, err := strconv.Atoi(s) if err != nil { - return cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) + return errors.WrapError(errors.ErrMySQLInvalidConfig, err) } if c < 0 { - return cerror.WrapError(cerror.ErrMySQLInvalidConfig, + return errors.WrapError(errors.ErrMySQLInvalidConfig, fmt.Errorf("invalid max-multi-update-row-size %d, "+ "which must be greater than or equal to 0", c)) } @@ -563,7 +564,7 @@ func (c *Config) getSSLCA(values url.Values, changefeedID common.ChangeFeedID, t name := fmt.Sprintf("cdc_mysql_tls%s_%s", changefeedID.Keyspace(), changefeedID.ID()) err = dmysql.RegisterTLSConfig(name, tlsCfg) if err != nil { - return cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") + return errors.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") } *tls = "?tls=" + name return nil @@ -590,13 +591,13 @@ func getTimezone(serverTimezone string, values url.Values, timezone *string) err changefeedTimezone, err := util.GetTimezone(s) if err != nil { - return cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) + return errors.WrapError(errors.ErrMySQLInvalidConfig, err) } *timezone = fmt.Sprintf(`"%s"`, changefeedTimezone.String()) // We need to check whether the timezone of the TiCDC server and the sink-uri are consistent. // If they are inconsistent, it may cause the data to be inconsistent. if changefeedTimezone.String() != serverTimezone { - return cerror.WrapError(cerror.ErrMySQLInvalidConfig, errors.Errorf( + return errors.WrapError(errors.ErrMySQLInvalidConfig, errors.Errorf( "the timezone of the TiCDC server and the sink-uri are inconsistent. "+ "TiCDC server timezone: %s, sink-uri timezone: %s. "+ "Please make sure that the timezone of the TiCDC server, "+ @@ -614,7 +615,7 @@ func getDuration(values url.Values, key string, target *string) error { } _, err := time.ParseDuration(s) if err != nil { - return cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) + return errors.WrapError(errors.ErrMySQLInvalidConfig, err) } *target = s return nil @@ -653,7 +654,7 @@ func getBool(values url.Values, key string, target *bool) error { if len(s) > 0 { enable, err := strconv.ParseBool(s) if err != nil { - return cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) + return errors.WrapError(errors.ErrMySQLInvalidConfig, err) } *target = enable } diff --git a/pkg/sink/mysql/helper.go b/pkg/sink/mysql/helper.go index f795d8dff1..58eaac2046 100644 --- a/pkg/sink/mysql/helper.go +++ b/pkg/sink/mysql/helper.go @@ -25,11 +25,10 @@ import ( "github.com/coreos/go-semver/semver" dmysql "github.com/go-sql-driver/mysql" - "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" commonEvent "github.com/pingcap/ticdc/pkg/common/event" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/retry" "github.com/pingcap/tidb/br/pkg/version" "github.com/pingcap/tidb/dumpling/export" @@ -70,8 +69,8 @@ func CheckIfBDRModeIsSupported(ctx context.Context, db *sql.DB) (bool, error) { query := "SET SESSION tidb_cdc_write_source = 1" _, err := db.ExecContext(ctx, query) if err != nil { - if mysqlErr, ok := errors.Cause(err).(*dmysql.MySQLError); ok && - mysqlErr.Number == mysql.ErrUnknownSystemVariable { + var mysqlErr *dmysql.MySQLError + if errors.As(errors.Cause(err), &mysqlErr) && mysqlErr.Number == mysql.ErrUnknownSystemVariable { return false, nil } return false, err @@ -189,7 +188,8 @@ func GetTestDB(dbConfig *dmysql.Config) (*sql.DB, error) { testDB, err := CreateMysqlDBConn(dbConfig.FormatDSN()) if err != nil { // If access is denied and password is encoded by base64, try to decoded password. - if mysqlErr, ok := errors.Cause(err).(*dmysql.MySQLError); ok && mysqlErr.Number == mysql.ErrAccessDenied { + var mysqlErr *dmysql.MySQLError + if errors.As(errors.Cause(err), &mysqlErr) && mysqlErr.Number == mysql.ErrAccessDenied { if dePassword, decodeErr := base64.StdEncoding.DecodeString(password); decodeErr == nil && string(dePassword) != password { dbConfig.Passwd = string(dePassword) testDB, err = CreateMysqlDBConn(dbConfig.FormatDSN()) @@ -204,9 +204,8 @@ func checkTiDBVariable(db *sql.DB, variableName, defaultValue string) (string, e var value string querySQL := fmt.Sprintf("show session variables like '%s';", variableName) err := db.QueryRowContext(context.Background(), querySQL).Scan(&name, &value) - if err != nil && err != sql.ErrNoRows { - errMsg := "fail to query session variable " + variableName - return "", cerror.ErrMySQLQueryError.Wrap(err).GenWithStack(errMsg) + if err != nil && !errors.Is(err, sql.ErrNoRows) { + return "", errors.WrapError(errors.ErrMySQLQueryError, err, "fail to query session variable %s", variableName) } // session variable works, use given default value if err == nil { @@ -309,10 +308,10 @@ func checkCharsetSupport(db *sql.DB, charsetName string) (bool, error) { querySQL := "select character_set_name from information_schema.character_sets " + "where character_set_name = '" + charsetName + "';" err = db.QueryRowContext(context.Background(), querySQL).Scan(&characterSetName) - if err != nil && err != sql.ErrNoRows { - return false, cerror.WrapError(cerror.ErrMySQLQueryError, err) + if err != nil && !errors.Is(err, sql.ErrNoRows) { + return false, errors.WrapError(errors.ErrMySQLQueryError, err) } - if err != nil { + if errors.Is(err, sql.ErrNoRows) { return false, nil } @@ -331,7 +330,9 @@ func GenerateDSN(ctx context.Context, cfg *Config) (string, error) { if err != nil { return "", err } - defer testDB.Close() + defer func() { + _ = testDB.Close() + }() // we use default sql mode for downstream because all dmls generated and ddls in ticdc // are based on default sql mode. @@ -344,7 +345,7 @@ func GenerateDSN(ctx context.Context, cfg *Config) (string, error) { cfg.IsTiDB = CheckIsTiDB(ctx, testDB) if cfg.EnableActiveActive && !cfg.IsTiDB { - return "", cerror.ErrMySQLInvalidConfig.GenWithStack( + return "", errors.ErrMySQLInvalidConfig.GenWithStack( "enable-active-active requires downstream TiDB") } @@ -391,7 +392,7 @@ func GenerateDSN(ctx context.Context, cfg *Config) (string, error) { func CreateMysqlDBConn(dsnStr string) (*sql.DB, error) { db, err := sql.Open("mysql", dsnStr) if err != nil { - return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") + return nil, errors.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") } err = db.PingContext(context.Background()) @@ -400,7 +401,7 @@ func CreateMysqlDBConn(dsnStr string) (*sql.DB, error) { if closeErr := db.Close(); closeErr != nil { log.Warn("close db failed", zap.Error(err)) } - return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") + return nil, errors.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") } return db, nil } @@ -452,7 +453,7 @@ func getCheckRunningAddIndexSQL(cfg *Config) string { } func isRetryableDMLError(err error) bool { - if !cerror.IsRetryableError(err) { + if !errors.IsRetryableError(err) { return false } @@ -470,7 +471,8 @@ func isRetryableDMLError(err error) bool { } func getSQLErrCode(err error) (errors.ErrCode, bool) { - mysqlErr, ok := errors.Cause(err).(*dmysql.MySQLError) + var mysqlErr *dmysql.MySQLError + ok := errors.As(errors.Cause(err), &mysqlErr) if !ok { return -1, false } @@ -484,7 +486,7 @@ func queryMaxPreparedStmtCount(ctx context.Context, db *sql.DB) (int, error) { var maxPreparedStmtCount sql.NullInt32 err := row.Scan(&maxPreparedStmtCount) if err != nil { - err = cerror.WrapError(cerror.ErrMySQLQueryError, err) + err = errors.WrapError(errors.ErrMySQLQueryError, err) } return int(maxPreparedStmtCount.Int32), err } @@ -494,7 +496,7 @@ func queryMaxAllowedPacket(ctx context.Context, db *sql.DB) (int64, error) { row := db.QueryRowContext(ctx, "select @@global.max_allowed_packet;") var maxAllowedPacket sql.NullInt64 if err := row.Scan(&maxAllowedPacket); err != nil { - return 0, cerror.WrapError(cerror.ErrMySQLQueryError, err) + return 0, errors.WrapError(errors.ErrMySQLQueryError, err) } return maxAllowedPacket.Int64, nil } diff --git a/pkg/sink/mysql/mysql_writer.go b/pkg/sink/mysql/mysql_writer.go index 6d78c54c0c..4c38fadaab 100644 --- a/pkg/sink/mysql/mysql_writer.go +++ b/pkg/sink/mysql/mysql_writer.go @@ -21,11 +21,10 @@ import ( "time" lru "github.com/hashicorp/golang-lru" - "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/ticdc/pkg/common" commonEvent "github.com/pingcap/ticdc/pkg/common/event" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/metrics" "go.uber.org/zap" ) @@ -260,7 +259,7 @@ func (w *Writer) checkIsDuplicateEntryError(err error) bool { if err == nil { return false } - if errors.Cause(err) == cerror.ErrMySQLDuplicateEntry || + if errors.Is(errors.Cause(err), errors.ErrMySQLDuplicateEntry) || strings.Contains(err.Error(), "Duplicate entry") { if !w.isInErrorCausedSafeMode { w.isInErrorCausedSafeMode = true diff --git a/pkg/sink/mysql/mysql_writer_ddl.go b/pkg/sink/mysql/mysql_writer_ddl.go index 235db4c465..6a7c6bf136 100644 --- a/pkg/sink/mysql/mysql_writer_ddl.go +++ b/pkg/sink/mysql/mysql_writer_ddl.go @@ -228,7 +228,7 @@ func (w *Writer) execDDLWithMaxRetries(event *commonEvent.DDLEvent) error { // If the error is ignorable, we will ignore the error directly. return nil } - if w.cfg.IsTiDB && ddlCreateTime != "" && errors.Cause(err) == mysql.ErrInvalidConn { + if w.cfg.IsTiDB && ddlCreateTime != "" && errors.Is(errors.Cause(err), mysql.ErrInvalidConn) { log.Warn("Wait the asynchronous ddl to synchronize", zap.String("ddl", event.Query), zap.String("ddlCreateTime", ddlCreateTime), zap.Uint64("startTs", event.GetStartTs()), zap.Uint64("commitTs", event.GetCommitTs()), zap.String("readTimeout", w.cfg.ReadTimeout), zap.Error(err)) diff --git a/pkg/sink/mysql/mysql_writer_dml_exec.go b/pkg/sink/mysql/mysql_writer_dml_exec.go index 8a25ceb236..8df25eaa78 100644 --- a/pkg/sink/mysql/mysql_writer_dml_exec.go +++ b/pkg/sink/mysql/mysql_writer_dml_exec.go @@ -22,10 +22,9 @@ import ( "time" dmysql "github.com/go-sql-driver/mysql" - "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/retry" "github.com/pingcap/ticdc/pkg/util" "github.com/pingcap/tidb/pkg/parser/mysql" @@ -35,7 +34,10 @@ import ( // execDMLWithMaxRetries executes prepared DMLs with retry/backoff handling. func (w *Writer) execDMLWithMaxRetries(dmls *preparedDMLs) error { if len(dmls.sqls) != len(dmls.values) || len(dmls.sqls) != len(dmls.rowTypes) { - return cerror.ErrUnexpected.FastGenByArgs(fmt.Sprintf("unexpected number of sqls and values or rowTypes, sqls is %s, values is %s, row types is %s", dmls.sqls, util.RedactAny(dmls.values), dmls.rowTypes)) + return errors.ErrUnexpected.FastGen( + "unexpected number of sqls and values or rowTypes, sqls is %s, values is %s, row types is %s", + dmls.sqls, util.RedactAny(dmls.values), dmls.rowTypes, + ) } // approximateSize is multiplied by 2 because in extreme circustumas, every @@ -93,7 +95,7 @@ func (w *Writer) execDMLWithMaxRetries(dmls *preparedDMLs) error { failpoint.Inject("MySQLSinkTxnRandomError", func() { log.Warn("inject MySQLSinkTxnRandomError") err := errors.Trace(driver.ErrBadConn) - w.logDMLTxnErr(err, time.Now(), w.ChangefeedID.String(), dmls) + err = w.logDMLTxnErr(err, time.Now(), w.ChangefeedID.String(), dmls) failpoint.Return(err) }) @@ -101,18 +103,17 @@ func (w *Writer) execDMLWithMaxRetries(dmls *preparedDMLs) error { failpoint.Inject("MySQLDuplicateEntryError", func() { log.Warn("inject MySQLDuplicateEntryError") - err := cerror.WrapError(cerror.ErrMySQLDuplicateEntry, &dmysql.MySQLError{ + err := errors.WrapError(errors.ErrMySQLDuplicateEntry, &dmysql.MySQLError{ Number: uint16(mysql.ErrDupEntry), Message: "Duplicate entry", }) - w.logDMLTxnErr(err, time.Now(), w.ChangefeedID.String(), dmls) + err = w.logDMLTxnErr(err, time.Now(), w.ChangefeedID.String(), dmls) failpoint.Return(err) }) err := w.statistics.RecordBatchExecution(tryExec) if err != nil { - w.logDMLTxnErr(err, time.Now(), w.ChangefeedID.String(), dmls) - return errors.Trace(err) + return errors.Trace(w.logDMLTxnErr(err, time.Now(), w.ChangefeedID.String(), dmls)) } return nil }, retry.WithBackoffBaseDelay(BackoffBaseDelay.Milliseconds()), @@ -158,12 +159,12 @@ func (w *Writer) sequenceExecute( if execError != nil { log.Error("ExecContext", zap.Error(execError), zap.Any("dmls", dmls), zap.Int("writerID", w.id)) if rbErr := tx.Rollback(); rbErr != nil { - if errors.Cause(rbErr) != context.Canceled { + if !errors.Is(errors.Cause(rbErr), context.Canceled) { log.Warn("failed to rollback txn", zap.Error(rbErr), zap.Int("writerID", w.id)) } } cancelFunc() - return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(execError, fmt.Sprintf("Failed to execute DMLs, query info:%s, args:%v; ", query, util.RedactArgs(args)))) + return errors.WrapError(errors.ErrMySQLTxnError, errors.WithMessage(execError, fmt.Sprintf("Failed to execute DMLs, query info:%s, args:%v; ", query, util.RedactArgs(args)))) } if rowsAffected, err := res.RowsAffected(); err != nil { log.Warn("get rows affected rows failed", zap.Error(err)) @@ -205,7 +206,7 @@ func (w *Writer) multiStmtExecute( zap.Int("writerID", w.id), zap.Error(rbErr)) } - return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, fmt.Sprintf("Failed to execute DMLs, query info:%s, args:%v; ", multiStmtSQLWithTxn, util.RedactArgs(multiStmtArgs)))) + return errors.WrapError(errors.ErrMySQLTxnError, errors.WithMessage(err, fmt.Sprintf("Failed to execute DMLs, query info:%s, args:%v; ", multiStmtSQLWithTxn, util.RedactArgs(multiStmtArgs)))) } if rowsAffected, err := res.RowsAffected(); err != nil { log.Warn("get rows affected rows failed", zap.Error(err)) diff --git a/pkg/sink/mysql/mysql_writer_for_active_active_sync_stats.go b/pkg/sink/mysql/mysql_writer_for_active_active_sync_stats.go index 5bbaefa916..072a9082ff 100644 --- a/pkg/sink/mysql/mysql_writer_for_active_active_sync_stats.go +++ b/pkg/sink/mysql/mysql_writer_for_active_active_sync_stats.go @@ -20,9 +20,9 @@ import ( "sync" dmysql "github.com/go-sql-driver/mysql" - "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/ticdc/pkg/common" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/metrics" tidbmysql "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/prometheus/client_golang/prometheus" @@ -41,8 +41,8 @@ func CheckActiveActiveSyncStatsSupported(ctx context.Context, db *sql.DB) (bool, row := db.QueryRowContext(ctx, "SELECT @@tidb_cdc_active_active_sync_stats;") var v sql.NullString if err := row.Scan(&v); err != nil { - if mysqlErr, ok := errors.Cause(err).(*dmysql.MySQLError); ok && - mysqlErr.Number == tidbmysql.ErrUnknownSystemVariable { + var mysqlErr *dmysql.MySQLError + if errors.As(errors.Cause(err), &mysqlErr) && mysqlErr.Number == tidbmysql.ErrUnknownSystemVariable { return false, nil } return false, errors.Trace(err) diff --git a/pkg/sink/mysql/mysql_writer_for_ddl_ts.go b/pkg/sink/mysql/mysql_writer_for_ddl_ts.go index d29e222ff5..423fce61fc 100644 --- a/pkg/sink/mysql/mysql_writer_for_ddl_ts.go +++ b/pkg/sink/mysql/mysql_writer_for_ddl_ts.go @@ -395,7 +395,9 @@ func (w *Writer) GetTableRecoveryInfo(tableIDs []int64) ([]int64, []bool, []bool } } - defer rows.Close() + defer func() { + _ = rows.Close() + }() type ddlTsRow struct { ddlTs int64 @@ -663,7 +665,9 @@ func (w *Writer) isDDLExecuted(tableID int64, ddlTs uint64) (bool, error) { return false, errors.WrapError(errors.ErrMySQLTxnError, errors.WithMessage(err, fmt.Sprintf("failed to check ddl ts table; Query is %s", query))) } - defer rows.Close() + defer func() { + _ = rows.Close() + }() if rows.Next() { return true, nil } diff --git a/pkg/sink/mysql/progress_table_writer.go b/pkg/sink/mysql/progress_table_writer.go index 2bb020cd8a..e192ecba37 100644 --- a/pkg/sink/mysql/progress_table_writer.go +++ b/pkg/sink/mysql/progress_table_writer.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/ticdc/pkg/common/event" "github.com/pingcap/ticdc/pkg/config" "github.com/pingcap/ticdc/pkg/errors" - cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/filter" "github.com/pingcap/ticdc/pkg/util" "go.uber.org/atomic" @@ -148,7 +147,7 @@ func (w *ProgressTableWriter) flushBatch( query := builder.String() _, err := w.db.ExecContext(w.ctx, query, args...) if err != nil { - return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, fmt.Sprintf("Failed to execute query, query info:%s, args:%v; ", query, util.RedactArgs(args)))) + return errors.WrapError(errors.ErrMySQLTxnError, errors.WithMessage(err, fmt.Sprintf("Failed to execute query, query info:%s, args:%v; ", query, util.RedactArgs(args)))) } return nil } @@ -165,7 +164,7 @@ func (w *ProgressTableWriter) initProgressTable(ctx context.Context) error { createDB := "CREATE DATABASE IF NOT EXISTS `" + filter.TiCDCSystemSchema + "`" if _, err := w.db.ExecContext(ctx, createDB); err != nil { - return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, fmt.Sprintf("Failed to execute sql, sql info:%s", createDB))) + return errors.WrapError(errors.ErrMySQLTxnError, errors.WithMessage(err, fmt.Sprintf("Failed to execute sql, sql info:%s", createDB))) } createTable := "CREATE TABLE IF NOT EXISTS `" + filter.TiCDCSystemSchema + "`.`" + progressTableName + "` (" + "changefeed_id VARCHAR(128) NOT NULL COMMENT 'Unique identifier for the changefeed synchronization task'," + @@ -176,7 +175,7 @@ func (w *ProgressTableWriter) initProgressTable(ctx context.Context) error { "PRIMARY KEY (changefeed_id, cluster_id, database_name, table_name)" + ") COMMENT='TiCDC synchronization progress table for HardDelete safety check'" if _, err := w.db.ExecContext(ctx, createTable); err != nil { - return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, fmt.Sprintf("Failed to execute sql, sql info:%s", createTable))) + return errors.WrapError(errors.ErrMySQLTxnError, errors.WithMessage(err, fmt.Sprintf("Failed to execute sql, sql info:%s", createTable))) } w.progressTableInit = true @@ -252,7 +251,7 @@ func (w *ProgressTableWriter) removeTableBatch(changefeed, clusterID string, tab query := builder.String() _, err := w.db.ExecContext(w.ctx, query, args...) if err != nil { - return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, fmt.Sprintf("Failed to execute query, query info:%s, args:%v; ", query, util.RedactArgs(args)))) + return errors.WrapError(errors.ErrMySQLTxnError, errors.WithMessage(err, fmt.Sprintf("Failed to execute query, query info:%s, args:%v; ", query, util.RedactArgs(args)))) } return nil } @@ -269,7 +268,7 @@ func (w *ProgressTableWriter) removeDatabase(changefeed, clusterID, dbName strin query := builder.String() _, err := w.db.ExecContext(w.ctx, query, changefeed, clusterID, dbName) if err != nil { - return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, fmt.Sprintf("Failed to execute query, query info:%s, args:%v; ", query, util.RedactArgs([]interface{}{changefeed, clusterID, dbName})))) + return errors.WrapError(errors.ErrMySQLTxnError, errors.WithMessage(err, fmt.Sprintf("Failed to execute query, query info:%s, args:%v; ", query, util.RedactArgs([]interface{}{changefeed, clusterID, dbName})))) } return nil } diff --git a/pkg/sink/mysql/progress_table_writer_test.go b/pkg/sink/mysql/progress_table_writer_test.go index ed572f3c04..197ae66489 100644 --- a/pkg/sink/mysql/progress_table_writer_test.go +++ b/pkg/sink/mysql/progress_table_writer_test.go @@ -21,14 +21,13 @@ import ( "github.com/DATA-DOG/go-sqlmock" "github.com/pingcap/ticdc/heartbeatpb" "github.com/pingcap/ticdc/pkg/common" - "github.com/pingcap/ticdc/pkg/common/event" commonEvent "github.com/pingcap/ticdc/pkg/common/event" "github.com/pingcap/ticdc/pkg/config" "github.com/pingcap/ticdc/pkg/filter" "github.com/stretchr/testify/require" ) -func newTestTableSchemaStore(tables []*event.SchemaTableName) *commonEvent.TableSchemaStore { +func newTestTableSchemaStore(tables []*commonEvent.SchemaTableName) *commonEvent.TableSchemaStore { schemaByName := make(map[string]*heartbeatpb.SchemaInfo) nextSchemaID := int64(1) nextTableID := int64(1) @@ -65,7 +64,7 @@ func TestProgressTableWriterFlushSingleBatch(t *testing.T) { setTestClusterID(t, "cluster-single") writer := NewProgressTableWriter(context.Background(), db, common.NewChangeFeedIDWithName("cf", "ks"), 10, 1*time.Millisecond) - tables := []*event.SchemaTableName{ + tables := []*commonEvent.SchemaTableName{ {SchemaName: "db1", TableName: "t1"}, {SchemaName: "db1", TableName: "t2"}, } @@ -88,7 +87,7 @@ func TestProgressTableWriterFlushMultiBatch(t *testing.T) { setTestClusterID(t, "cluster-multi") writer := NewProgressTableWriter(context.Background(), db, common.NewChangeFeedIDWithName("cf", "ks"), 2, 1*time.Millisecond) - allTables := []*event.SchemaTableName{ + allTables := []*commonEvent.SchemaTableName{ {SchemaName: "db1", TableName: "t1"}, {SchemaName: "db1", TableName: "t2"}, {SchemaName: "db1", TableName: "t3"}, @@ -121,7 +120,7 @@ func expectProgressTableInit(mock sqlmock.Sqlmock) { WillReturnResult(sqlmock.NewResult(0, 0)) } -func expectProgressInsert(mock sqlmock.Sqlmock, changefeed, cluster string, checkpoint uint64, tables []*event.SchemaTableName) { +func expectProgressInsert(mock sqlmock.Sqlmock, changefeed, cluster string, checkpoint uint64, tables []*commonEvent.SchemaTableName) { args := make([]driver.Value, 0, len(tables)*5) for _, tbl := range tables { // the tables order is not guaranteed, so we only check the other field in the args. diff --git a/pkg/sink/mysql/sql_builder.go b/pkg/sink/mysql/sql_builder.go index 17aeaf0b5a..562b42de35 100644 --- a/pkg/sink/mysql/sql_builder.go +++ b/pkg/sink/mysql/sql_builder.go @@ -122,7 +122,7 @@ func (d *preparedDMLs) RowsAffected() int64 { for _, rowType := range d.rowTypes { switch rowType { case common.RowTypeInsert, common.RowTypeDelete: - count += 1 + count++ case common.RowTypeUpdate: count += 2 default: diff --git a/pkg/sink/mysql/sql_builder_test.go b/pkg/sink/mysql/sql_builder_test.go index c88dc06830..78924e32f6 100644 --- a/pkg/sink/mysql/sql_builder_test.go +++ b/pkg/sink/mysql/sql_builder_test.go @@ -17,63 +17,62 @@ import ( "testing" "github.com/pingcap/ticdc/pkg/common" - "github.com/pingcap/ticdc/pkg/common/event" - pevent "github.com/pingcap/ticdc/pkg/common/event" + commonEvent "github.com/pingcap/ticdc/pkg/common/event" "github.com/stretchr/testify/require" ) // This table has 45 columns var preCreateTableSQL = `create table t ( id int primary key auto_increment, - + c_tinyint tinyint null, c_smallint smallint null, c_mediumint mediumint null, c_int int null, c_bigint bigint null, - + c_unsigned_tinyint tinyint unsigned null, c_unsigned_smallint smallint unsigned null, c_unsigned_mediumint mediumint unsigned null, c_unsigned_int int unsigned null, c_unsigned_bigint bigint unsigned null, - + c_float float null, c_double double null, c_decimal decimal null, c_decimal_2 decimal(10, 4) null, - + c_unsigned_float float unsigned null, c_unsigned_double double unsigned null, c_unsigned_decimal decimal unsigned null, c_unsigned_decimal_2 decimal(10, 4) unsigned null, - + c_date date null, c_datetime datetime null, c_timestamp timestamp null, c_time time null, c_year year null, - + c_tinytext tinytext null, c_text text null, c_mediumtext mediumtext null, c_longtext longtext null, - + c_tinyblob tinyblob null, c_blob blob null, c_mediumblob mediumblob null, c_longblob longblob null, - + c_char char(16) null, c_varchar varchar(16) null, c_binary binary(16) null, c_varbinary varbinary(16) null, - + c_enum enum ('a','b','c') null, c_set set ('a','b','c') null, c_bit bit(64) null, c_json json null, - + -- gbk dmls name varchar(128) CHARACTER SET gbk, country char(32) CHARACTER SET gbk, @@ -101,8 +100,8 @@ var preInsertDataSQL = `insert into t values ( '测试', "中国", "上海", "你好,世界", 0xC4E3BAC3CAC0BDE7 );` -func getRowForTest(t testing.TB) (insert, delete, update pevent.RowChange, tableInfo *common.TableInfo) { - helper := pevent.NewEventTestHelper(t) +func getRowForTest(t testing.TB) (insert, delete, update commonEvent.RowChange, tableInfo *common.TableInfo) { + helper := commonEvent.NewEventTestHelper(t) defer helper.Close() helper.Tk().MustExec("use test") @@ -123,7 +122,7 @@ func getRowForTest(t testing.TB) (insert, delete, update pevent.RowChange, table update.PreRow = insert.Row update.RowType = common.RowTypeUpdate - delete = pevent.RowChange{ + delete = commonEvent.RowChange{ PreRow: insert.Row, RowType: common.RowTypeDelete, } @@ -183,7 +182,7 @@ func TestBuildInsert(t *testing.T) { } func TestBuildDelete(t *testing.T) { - helper := event.NewEventTestHelper(t) + helper := commonEvent.NewEventTestHelper(t) defer helper.Close() helper.Tk().MustExec("use test") @@ -301,7 +300,7 @@ func TestBuildDelete(t *testing.T) { } func TestBuildUpdate(t *testing.T) { - helper := event.NewEventTestHelper(t) + helper := commonEvent.NewEventTestHelper(t) defer helper.Close() helper.Tk().MustExec("use test") diff --git a/pkg/tcpserver/tcp_server.go b/pkg/tcpserver/tcp_server.go index d02a5f326d..fa4640a5e1 100644 --- a/pkg/tcpserver/tcp_server.go +++ b/pkg/tcpserver/tcp_server.go @@ -20,9 +20,8 @@ import ( "strings" "time" - "github.com/pingcap/errors" "github.com/pingcap/log" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/security" "github.com/soheilhy/cmux" "go.uber.org/atomic" @@ -109,7 +108,7 @@ func NewTCPServer(address string, credentials *security.Credential) (TCPServer, // Run runs the mux. The mux has to be running to accept connections. func (s *tcpServerImpl) Run(ctx context.Context) (err error) { if s.isClosed.Load() { - return cerror.ErrTCPServerClosed.GenWithStackByArgs() + return errors.ErrTCPServerClosed.GenWithStackByArgs() } log.Info("tcp server start to serve") defer func() { @@ -127,11 +126,11 @@ func (s *tcpServerImpl) Run(ctx context.Context) (err error) { errg.Go(func() error { err := s.mux.Serve() - if err == cmux.ErrServerClosed { - return cerror.ErrTCPServerClosed.GenWithStackByArgs() + if errors.Is(err, cmux.ErrServerClosed) { + return errors.ErrTCPServerClosed.GenWithStackByArgs() } if err != nil && strings.Contains(err.Error(), "use of closed network connection") { - return cerror.ErrTCPServerClosed.GenWithStackByArgs() + return errors.ErrTCPServerClosed.GenWithStackByArgs() } return errors.Trace(err) }) diff --git a/pkg/txnutil/gc/testing.go b/pkg/txnutil/gc/testing.go index accf59e7d2..b6353db871 100644 --- a/pkg/txnutil/gc/testing.go +++ b/pkg/txnutil/gc/testing.go @@ -40,7 +40,7 @@ func (m *MockPDClient) UpdateServiceGCSafePoint(ctx context.Context, serviceID s } // GetTS implements pd.Client.GetTS. -func (m *MockPDClient) GetTS(ctx context.Context) (int64, int64, error) { +func (m *MockPDClient) GetTS(_ context.Context) (int64, int64, error) { return oracle.GetPhysical(time.Now()), 0, nil } @@ -49,7 +49,7 @@ func (m *MockPDClient) GetTS(ctx context.Context) (int64, int64, error) { func (m *MockPDClient) Close() {} // GetClusterID gets the cluster ID from PD. -func (m *MockPDClient) GetClusterID(ctx context.Context) uint64 { +func (m *MockPDClient) GetClusterID(_ context.Context) uint64 { return m.ClusterID } @@ -62,8 +62,8 @@ func (m *MockPDClient) GetAllStores( // LoadGlobalConfig loads global config from PD. func (m *MockPDClient) LoadGlobalConfig( - ctx context.Context, - names []string, configPath string, + _ context.Context, + _ []string, _ string, ) ([]pd.GlobalConfigItem, int64, error) { return []pd.GlobalConfigItem{ { diff --git a/pkg/upstream/manager.go b/pkg/upstream/manager.go index 29b1dffbea..09a5b678a9 100644 --- a/pkg/upstream/manager.go +++ b/pkg/upstream/manager.go @@ -20,7 +20,7 @@ import ( "github.com/benbjohnson/clock" "github.com/pingcap/log" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/etcd" "github.com/pingcap/ticdc/pkg/node" "github.com/pingcap/ticdc/pkg/security" @@ -89,7 +89,7 @@ func (m *Manager) AddDefaultUpstream( if err := m.initUpstreamFunc(m.ctx, up, &m.nodeCfg); err != nil { up.err.Store(err) up.Close() - return nil, cerror.Trace(err) + return nil, errors.Trace(err) } m.defaultUpstream = up m.ups.Store(up.ID, up) @@ -100,7 +100,7 @@ func (m *Manager) AddDefaultUpstream( // GetDefaultUpstream returns the default upstream func (m *Manager) GetDefaultUpstream() (*Upstream, error) { if m.defaultUpstream == nil { - return nil, cerror.ErrUpstreamNotFound + return nil, errors.ErrUpstreamNotFound } return m.defaultUpstream, nil } @@ -186,7 +186,7 @@ func (m *Manager) Close() { // Visit on each upstream, return error on the first func (m *Manager) Visit(visitor func(up *Upstream) error) error { var err error - m.ups.Range(func(k, v interface{}) bool { + m.ups.Range(func(_, v interface{}) bool { err = visitor(v.(*Upstream)) return err == nil }) diff --git a/pkg/util/atomic.go b/pkg/util/atomic.go index a0047a83fc..ba77b63ca8 100644 --- a/pkg/util/atomic.go +++ b/pkg/util/atomic.go @@ -20,18 +20,18 @@ type numbers interface { type genericAtomic[T numbers] interface { Load() T Store(T) - CompareAndSwap(old, new T) bool + CompareAndSwap(old, newVal T) bool } // CompareAndIncrease updates the target if the new value is larger than or equal to the old value. // It returns false if the new value is smaller than the old value. -func CompareAndIncrease[T numbers](target genericAtomic[T], new T) bool { +func CompareAndIncrease[T numbers](target genericAtomic[T], newVal T) bool { for { old := target.Load() - if new < old { + if newVal < old { return false } - if new == old || target.CompareAndSwap(old, new) { + if newVal == old || target.CompareAndSwap(old, newVal) { return true } } @@ -39,13 +39,13 @@ func CompareAndIncrease[T numbers](target genericAtomic[T], new T) bool { // CompareAndMonotonicIncrease updates the target if the new value is larger than the old value. // It returns false if the new value is smaller than or equal to the old value. -func CompareAndMonotonicIncrease[T numbers](target genericAtomic[T], new T) bool { +func CompareAndMonotonicIncrease[T numbers](target genericAtomic[T], newVal T) bool { for { old := target.Load() - if new <= old { + if newVal <= old { return false } - if target.CompareAndSwap(old, new) { + if target.CompareAndSwap(old, newVal) { return true } } @@ -53,6 +53,6 @@ func CompareAndMonotonicIncrease[T numbers](target genericAtomic[T], new T) bool // MustCompareAndMonotonicIncrease updates the target if the new value is larger than the old value. It do nothing // if the new value is smaller than or equal to the old value. -func MustCompareAndMonotonicIncrease[T numbers](target genericAtomic[T], new T) { - _ = CompareAndMonotonicIncrease(target, new) +func MustCompareAndMonotonicIncrease[T numbers](target genericAtomic[T], newVal T) { + _ = CompareAndMonotonicIncrease(target, newVal) } diff --git a/pkg/util/json_writer.go b/pkg/util/json_writer.go index f696e135fe..cf17ec744d 100644 --- a/pkg/util/json_writer.go +++ b/pkg/util/json_writer.go @@ -71,7 +71,7 @@ func BorrowJSONWriter(out io.Writer) *JSONWriter { // ReturnJSONWriter returns the borrowed JSONWriter instance to pool. func ReturnJSONWriter(w *JSONWriter) { - w.stream.Flush() + _ = w.stream.Flush() jsonAPI.ReturnStream(w.stream) w.out = nil w.stream = nil diff --git a/pkg/util/session.go b/pkg/util/session.go index f123d3d1d7..9c7aa7d737 100644 --- a/pkg/util/session.go +++ b/pkg/util/session.go @@ -78,7 +78,7 @@ func (se *session) GetBuiltinFunctionUsage() map[string]uint32 { return se.builtinFunctionUsage } -func (se *session) BuiltinFunctionUsageInc(scalarFuncSigName string) {} +func (se *session) BuiltinFunctionUsageInc(_ string) {} // ZeroSessionCtx is used when the session variables is not important. var ZeroSessionCtx sessionctx.Context diff --git a/pkg/version/check.go b/pkg/version/check.go index 703d9d36a6..70b3aaab12 100644 --- a/pkg/version/check.go +++ b/pkg/version/check.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/pingcap/ticdc/pkg/errors" - cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/httputil" "github.com/pingcap/ticdc/pkg/retry" "github.com/pingcap/ticdc/pkg/security" @@ -99,7 +98,7 @@ func CheckClusterVersion( }, retry.WithBackoffBaseDelay(time.Millisecond.Milliseconds()*10), retry.WithBackoffMaxDelay(time.Second.Milliseconds()), retry.WithMaxTries(uint64(checkClusterVersionRetryTimes)), - retry.WithIsRetryableErr(cerror.IsRetryableError)) + retry.WithIsRetryableErr(errors.IsRetryableError)) if err == nil { break } @@ -118,23 +117,23 @@ func CheckTiCDCVersion(versions map[string]struct{}) error { if len(versions) >= 3 { arg := fmt.Sprintf("all running cdc instance belong to %d different versions, "+ "it's not allowed", len(versions)) - return cerror.ErrVersionIncompatible.GenWithStackByArgs(arg) + return errors.ErrVersionIncompatible.GenWithStackByArgs(arg) } ver := &semver.Version{} for v := range versions { if err := ver.Set(SanitizeVersion(v)); err != nil { - return cerror.WrapError(cerror.ErrNewSemVersion, err) + return errors.WrapError(errors.ErrNewSemVersion, err) } if ver.Compare(*MinTiCDCVersion) < 0 { arg := fmt.Sprintf("TiCDC %s is not supported, the minimal compatible version is %s", SanitizeVersion(v), MinTiCDCVersion) - return cerror.ErrVersionIncompatible.GenWithStackByArgs(arg) + return errors.ErrVersionIncompatible.GenWithStackByArgs(arg) } if ver.Compare(*MaxTiCDCVersion) >= 0 { arg := fmt.Sprintf("TiCDC %s is not supported, only support version less than %s", SanitizeVersion(v), MaxTiCDCVersion) - return cerror.ErrVersionIncompatible.GenWithStackByArgs(arg) + return errors.ErrVersionIncompatible.GenWithStackByArgs(arg) } } return nil @@ -156,9 +155,11 @@ func checkPDVersion(ctx context.Context, pdAddr string, credential *security.Cre defer cancel() resp, err := httpClient.Get(ctx, fmt.Sprintf("%s/pd/api/v1/version", pdAddr)) if err != nil { - return cerror.ErrCheckClusterVersionFromPD.GenWithStackByArgs(err) + return errors.ErrCheckClusterVersionFromPD.GenWithStackByArgs(err) } - defer resp.Body.Close() + defer func() { + _ = resp.Body.Close() + }() content, err := io.ReadAll(resp.Body) if err != nil || resp.StatusCode < 200 || resp.StatusCode >= 300 { @@ -168,31 +169,31 @@ func checkPDVersion(ctx context.Context, pdAddr string, credential *security.Cre } else { arg = fmt.Sprintf("%s %s", resp.Status, content) } - return cerror.ErrCheckClusterVersionFromPD.GenWithStackByArgs(arg) + return errors.ErrCheckClusterVersionFromPD.GenWithStackByArgs(arg) } err = json.Unmarshal(content, &pdVer) if err != nil { - return cerror.ErrCheckClusterVersionFromPD.GenWithStackByArgs(err) + return errors.ErrCheckClusterVersionFromPD.GenWithStackByArgs(err) } ver, err := semver.NewVersion(SanitizeVersion(pdVer.Version)) if err != nil { err = errors.Annotate(err, "invalid PD version") - return cerror.WrapError(cerror.ErrNewSemVersion, err) + return errors.WrapError(errors.ErrNewSemVersion, err) } minOrd := ver.Compare(*minPDVersion) if minOrd < 0 { arg := fmt.Sprintf("PD %s is not supported, the minimal compatible version is %s", SanitizeVersion(pdVer.Version), minPDVersion) - return cerror.ErrVersionIncompatible.GenWithStackByArgs(arg) + return errors.ErrVersionIncompatible.GenWithStackByArgs(arg) } maxOrd := ver.Compare(*maxPDVersion) if maxOrd >= 0 { arg := fmt.Sprintf("PD %s is not supported, only support version less than %s", SanitizeVersion(pdVer.Version), maxPDVersion) - return cerror.ErrVersionIncompatible.GenWithStackByArgs(arg) + return errors.ErrVersionIncompatible.GenWithStackByArgs(arg) } return nil } @@ -201,11 +202,11 @@ func checkPDVersion(ctx context.Context, pdAddr string, credential *security.Cre // If storeID is 0, it checks all TiKV. func CheckStoreVersion(ctx context.Context, client pd.Client) error { failpoint.Inject("GetStoreFailed", func() { - failpoint.Return(cerror.WrapError(cerror.ErrGetAllStoresFailed, fmt.Errorf("unknown store"))) + failpoint.Return(errors.WrapError(errors.ErrGetAllStoresFailed, fmt.Errorf("unknown store"))) }) stores, err := client.GetAllStores(ctx, pdopt.WithExcludeTombstone()) if err != nil { - return cerror.WrapError(cerror.ErrGetAllStoresFailed, err) + return errors.WrapError(errors.ErrGetAllStoresFailed, err) } for _, s := range stores { @@ -216,19 +217,19 @@ func CheckStoreVersion(ctx context.Context, client pd.Client) error { ver, err := semver.NewVersion(SanitizeVersion(s.Version)) if err != nil { err = errors.Annotate(err, "invalid TiKV version") - return cerror.WrapError(cerror.ErrNewSemVersion, err) + return errors.WrapError(errors.ErrNewSemVersion, err) } minOrd := ver.Compare(*MinTiKVVersion) if minOrd < 0 { arg := fmt.Sprintf("TiKV %s is not supported, the minimal compatible version is %s", SanitizeVersion(s.Version), MinTiKVVersion) - return cerror.ErrVersionIncompatible.GenWithStackByArgs(arg) + return errors.ErrVersionIncompatible.GenWithStackByArgs(arg) } maxOrd := ver.Compare(*maxTiKVVersion) if maxOrd >= 0 { arg := fmt.Sprintf("TiKV %s is not supported, only support version less than %s", SanitizeVersion(s.Version), maxTiKVVersion) - return cerror.ErrVersionIncompatible.GenWithStackByArgs(arg) + return errors.ErrVersionIncompatible.GenWithStackByArgs(arg) } } return nil @@ -268,7 +269,7 @@ func GetTiCDCClusterVersion(captureVersion []string) (TiCDCClusterVersion, error } if err != nil { err = errors.Annotate(err, "invalid CDC cluster version") - return ticdcClusterVersionUnknown, cerror.WrapError(cerror.ErrNewSemVersion, err) + return ticdcClusterVersionUnknown, errors.WrapError(errors.ErrNewSemVersion, err) } if minVer == nil || ver.Compare(*minVer) < 0 { minVer = ver diff --git a/tests/integration_tests/bank/case.go b/tests/integration_tests/bank/case.go index 4437afb37d..81ca200755 100644 --- a/tests/integration_tests/bank/case.go +++ b/tests/integration_tests/bank/case.go @@ -29,9 +29,8 @@ import ( "time" _ "github.com/go-sql-driver/mysql" // MySQL driver - "github.com/pingcap/errors" "github.com/pingcap/log" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/retry" "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" @@ -194,7 +193,7 @@ func (*sequenceTest) verify(ctx context.Context, db *sql.DB, accounts, tableID i } return nil - }, retry.WithBackoffMaxDelay(500), retry.WithBackoffMaxDelay(120*1000), retry.WithMaxTries(10), retry.WithIsRetryableErr(cerror.IsRetryableError)) + }, retry.WithBackoffMaxDelay(500), retry.WithBackoffMaxDelay(120*1000), retry.WithMaxTries(10), retry.WithIsRetryableErr(errors.IsRetryableError)) } // tryDropDB will drop table if data incorrect and panic error likes bad connect. @@ -320,7 +319,7 @@ func (*bankTest) verify(ctx context.Context, db *sql.DB, accounts, tableID int, } return nil - }, retry.WithBackoffBaseDelay(int64(retryInterval)), retry.WithBackoffMaxDelay(120*1000), retry.WithMaxTries(20), retry.WithIsRetryableErr(cerror.IsRetryableError)) + }, retry.WithBackoffBaseDelay(int64(retryInterval)), retry.WithBackoffMaxDelay(120*1000), retry.WithMaxTries(20), retry.WithIsRetryableErr(errors.IsRetryableError)) } // tryDropDB will drop table if data incorrect and panic error likes bad connect. @@ -561,7 +560,7 @@ func run( return ctx.Err() default: err := workload() - if err != nil && errors.Cause(err) != context.Canceled { + if err != nil && !errors.Is(errors.Cause(err), context.Canceled) { log.Warn("workload failed", zap.Error(err)) } diff --git a/tests/integration_tests/many_pk_or_uk/main.go b/tests/integration_tests/many_pk_or_uk/main.go index 43bed77a99..0bd5cdbe0d 100644 --- a/tests/integration_tests/many_pk_or_uk/main.go +++ b/tests/integration_tests/many_pk_or_uk/main.go @@ -21,17 +21,17 @@ import ( "os" "sync" - "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/tests/integration_tests/util" ) func main() { cfg := util.NewConfig() err := cfg.Parse(os.Args[1:]) - switch errors.Cause(err) { - case nil: - case flag.ErrHelp: + switch { + case err == nil: + case errors.Is(err, flag.ErrHelp): os.Exit(0) default: log.S().Errorf("parse cmd flags err %s\n", err) @@ -57,8 +57,8 @@ func main() { func runPKorUKcases(db *sql.DB) { cases := []struct { Tp string - Value interface{} - Update interface{} + Value any + Update any }{ { Tp: "BIGINT UNSIGNED", diff --git a/utils/threadpool/thread_pool_test_task.go b/utils/threadpool/thread_pool_test_task.go index 40d51811d5..09fc0c3a44 100644 --- a/utils/threadpool/thread_pool_test_task.go +++ b/utils/threadpool/thread_pool_test_task.go @@ -60,15 +60,6 @@ type PureCPUTask struct { addCount int } -func newPureCPUTask(finalChan *chan int, ch *chan int, target int64, addCount int) *PureCPUTask { - return &PureCPUTask{ - finalChan: finalChan, - ch: ch, - target: target, - addCount: addCount, - } -} - func (t *PureCPUTask) Execute() time.Time { select { case <-*t.ch: