From 9a6bd36145833da144821884461e2e0d8b13c2d1 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Fri, 5 Nov 2021 17:33:04 +0800 Subject: [PATCH 1/5] dm-relay/streamer: move binlog log reader into relay (#3243) --- dm/dm/worker/relay.go | 17 +- dm/dm/worker/relay_test.go | 7 +- dm/dm/worker/server_test.go | 4 +- dm/dm/worker/source_worker.go | 6 +- dm/dm/worker/source_worker_test.go | 6 +- dm/dm/worker/subtask.go | 6 +- dm/dm/worker/subtask_test.go | 12 +- dm/pkg/binlog/writer/writer.go | 32 -- dm/pkg/streamer/util.go | 55 --- dm/pkg/streamer/util_test.go | 124 ------ .../writer/file.go => relay/binlog_writer.go} | 38 +- .../binlog_writer_test.go} | 20 +- dm/relay/config.go | 5 +- dm/relay/{reader => }/error.go | 2 +- dm/relay/{reader => }/error_test.go | 2 +- dm/{pkg/streamer => relay}/file.go | 4 +- dm/{pkg/streamer => relay}/file_test.go | 6 +- dm/relay/{writer => }/file_util.go | 2 +- dm/relay/{writer => }/file_util_test.go | 68 +--- .../reader.go => relay/local_reader.go} | 14 +- .../local_reader_test.go} | 21 +- dm/relay/purge_strategy.go | 360 ++++++++++++++++++ dm/relay/{purger => }/purger.go | 36 +- dm/relay/purger/strategy.go | 67 ---- dm/relay/purger/strategy_filename.go | 118 ------ dm/relay/purger/strategy_inactive.go | 86 ----- dm/relay/purger/strategy_space.go | 96 ----- dm/relay/purger/strategy_time.go | 88 ----- dm/relay/{purger/file.go => purger_helper.go} | 6 +- .../file_test.go => purger_helper_test.go} | 2 +- dm/relay/{purger => }/purger_test.go | 18 +- dm/relay/relay.go | 29 +- dm/relay/relay_test.go | 24 +- dm/relay/{writer/file.go => relay_writer.go} | 114 ++++-- .../file_test.go => relay_writer_test.go} | 4 +- dm/relay/{retry/reader.go => remote_retry.go} | 2 +- .../reader_test.go => remote_retry_test.go} | 4 +- dm/{pkg/streamer => relay}/streamer.go | 6 +- dm/{pkg/streamer => relay}/streamer_test.go | 10 +- .../{reader/reader.go => upstream_reader.go} | 34 +- ...reader_test.go => upstream_reader_test.go} | 28 +- dm/relay/util.go | 35 ++ dm/relay/util_test.go | 97 +++++ dm/relay/writer/writer.go | 73 ---- dm/syncer/streamer_controller.go | 19 +- dm/syncer/syncer.go | 7 +- dm/syncer/syncer_test.go | 4 +- dm/tests/dmctl_basic/run.sh | 2 +- dm/tests/duplicate_event/run.sh | 2 +- dm/tests/only_dml/run.sh | 2 +- dm/tests/relay_interrupt/run.sh | 2 +- dm/tests/sharding2/run.sh | 2 +- 52 files changed, 771 insertions(+), 1057 deletions(-) delete mode 100644 dm/pkg/binlog/writer/writer.go delete mode 100644 dm/pkg/streamer/util.go delete mode 100644 dm/pkg/streamer/util_test.go rename dm/{pkg/binlog/writer/file.go => relay/binlog_writer.go} (80%) rename dm/{pkg/binlog/writer/file_test.go => relay/binlog_writer_test.go} (87%) rename dm/relay/{reader => }/error.go (98%) rename dm/relay/{reader => }/error_test.go (98%) rename dm/{pkg/streamer => relay}/file.go (99%) rename dm/{pkg/streamer => relay}/file_test.go (99%) rename dm/relay/{writer => }/file_util.go (99%) rename dm/relay/{writer => }/file_util_test.go (90%) rename dm/{pkg/streamer/reader.go => relay/local_reader.go} (97%) rename dm/{pkg/streamer/reader_test.go => relay/local_reader_test.go} (98%) create mode 100644 dm/relay/purge_strategy.go rename dm/relay/{purger => }/purger.go (90%) delete mode 100644 dm/relay/purger/strategy.go delete mode 100644 dm/relay/purger/strategy_filename.go delete mode 100644 dm/relay/purger/strategy_inactive.go delete mode 100644 dm/relay/purger/strategy_space.go delete mode 100644 dm/relay/purger/strategy_time.go rename dm/relay/{purger/file.go => purger_helper.go} (97%) rename dm/relay/{purger/file_test.go => purger_helper_test.go} (99%) rename dm/relay/{purger => }/purger_test.go (95%) rename dm/relay/{writer/file.go => relay_writer.go} (77%) rename dm/relay/{writer/file_test.go => relay_writer_test.go} (99%) rename dm/relay/{retry/reader.go => remote_retry.go} (99%) rename dm/relay/{retry/reader_test.go => remote_retry_test.go} (97%) rename dm/{pkg/streamer => relay}/streamer.go (94%) rename dm/{pkg/streamer => relay}/streamer_test.go (86%) rename dm/relay/{reader/reader.go => upstream_reader.go} (85%) rename dm/relay/{reader/reader_test.go => upstream_reader_test.go} (84%) delete mode 100644 dm/relay/writer/writer.go diff --git a/dm/dm/worker/relay.go b/dm/dm/worker/relay.go index a4ff098ecad..b0ccc934293 100644 --- a/dm/dm/worker/relay.go +++ b/dm/dm/worker/relay.go @@ -28,13 +28,12 @@ import ( "github.com/pingcap/ticdc/dm/pkg/streamer" "github.com/pingcap/ticdc/dm/pkg/terror" "github.com/pingcap/ticdc/dm/relay" - "github.com/pingcap/ticdc/dm/relay/purger" ) // RelayHolder for relay unit. type RelayHolder interface { // Init initializes the holder - Init(ctx context.Context, interceptors []purger.PurgeInterceptor) (purger.Purger, error) + Init(ctx context.Context, interceptors []relay.PurgeInterceptor) (relay.Purger, error) // Start starts run the relay Start() // Close closes the holder @@ -94,11 +93,11 @@ func NewRealRelayHolder(sourceCfg *config.SourceConfig) RelayHolder { } // Init initializes the holder. -func (h *realRelayHolder) Init(ctx context.Context, interceptors []purger.PurgeInterceptor) (purger.Purger, error) { +func (h *realRelayHolder) Init(ctx context.Context, interceptors []relay.PurgeInterceptor) (relay.Purger, error) { h.closed.Store(false) // initial relay purger - operators := []purger.RelayOperator{ + operators := []relay.Operator{ h, streamer.GetReaderHub(), } @@ -107,7 +106,7 @@ func (h *realRelayHolder) Init(ctx context.Context, interceptors []purger.PurgeI return nil, terror.Annotate(err, "initial relay unit") } - return purger.NewPurger(h.cfg.Purge, h.cfg.RelayDir, operators, interceptors), nil + return relay.NewPurger(h.cfg.Purge, h.cfg.RelayDir, operators, interceptors), nil } // Start starts run the relay. @@ -306,7 +305,7 @@ func (h *realRelayHolder) Update(ctx context.Context, sourceCfg *config.SourceCo return nil } -// EarliestActiveRelayLog implements RelayOperator.EarliestActiveRelayLog. +// EarliestActiveRelayLog implements Operator.EarliestActiveRelayLog. func (h *realRelayHolder) EarliestActiveRelayLog() *streamer.RelayLogInfo { return h.relay.ActiveRelayLog() } @@ -355,13 +354,13 @@ func NewDummyRelayHolderWithInitError(cfg *config.SourceConfig) RelayHolder { } // Init implements interface of RelayHolder. -func (d *dummyRelayHolder) Init(ctx context.Context, interceptors []purger.PurgeInterceptor) (purger.Purger, error) { +func (d *dummyRelayHolder) Init(ctx context.Context, interceptors []relay.PurgeInterceptor) (relay.Purger, error) { // initial relay purger - operators := []purger.RelayOperator{ + operators := []relay.Operator{ d, } - return purger.NewDummyPurger(d.cfg.Purge, d.cfg.RelayDir, operators, interceptors), d.initError + return relay.NewDummyPurger(d.cfg.Purge, d.cfg.RelayDir, operators, interceptors), d.initError } // Start implements interface of RelayHolder. diff --git a/dm/dm/worker/relay_test.go b/dm/dm/worker/relay_test.go index 6c447c40048..bbf12856de6 100644 --- a/dm/dm/worker/relay_test.go +++ b/dm/dm/worker/relay_test.go @@ -29,7 +29,6 @@ import ( pkgstreamer "github.com/pingcap/ticdc/dm/pkg/streamer" "github.com/pingcap/ticdc/dm/pkg/utils" "github.com/pingcap/ticdc/dm/relay" - "github.com/pingcap/ticdc/dm/relay/purger" ) type testRelay struct{} @@ -139,11 +138,11 @@ func (d *DummyRelay) PurgeRelayDir() error { func (t *testRelay) TestRelay(c *C) { originNewRelay := relay.NewRelay relay.NewRelay = NewDummyRelay - originNewPurger := purger.NewPurger - purger.NewPurger = purger.NewDummyPurger + originNewPurger := relay.NewPurger + relay.NewPurger = relay.NewDummyPurger defer func() { relay.NewRelay = originNewRelay - purger.NewPurger = originNewPurger + relay.NewPurger = originNewPurger }() cfg := loadSourceConfigWithoutPassword(c) diff --git a/dm/dm/worker/server_test.go b/dm/dm/worker/server_test.go index 20d620c8e3f..b9470f34634 100644 --- a/dm/dm/worker/server_test.go +++ b/dm/dm/worker/server_test.go @@ -38,9 +38,9 @@ import ( "github.com/pingcap/ticdc/dm/pkg/gtid" "github.com/pingcap/ticdc/dm/pkg/ha" "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/streamer" "github.com/pingcap/ticdc/dm/pkg/terror" "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/ticdc/dm/relay" ) // do not forget to update this path if the file removed/renamed. @@ -121,7 +121,7 @@ func (t *testServer) TestServer(c *C) { cfg.UseRelay = false return NewRealSubTask(cfg, etcdClient, worker) } - createUnits = func(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, worker string, notifier streamer.EventNotifier) []unit.Unit { + createUnits = func(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, worker string, notifier relay.EventNotifier) []unit.Unit { mockDumper := NewMockUnit(pb.UnitType_Dump) mockLoader := NewMockUnit(pb.UnitType_Load) mockSync := NewMockUnit(pb.UnitType_Sync) diff --git a/dm/dm/worker/source_worker.go b/dm/dm/worker/source_worker.go index 9ff7097830d..d2fdf80aaf2 100644 --- a/dm/dm/worker/source_worker.go +++ b/dm/dm/worker/source_worker.go @@ -37,7 +37,7 @@ import ( "github.com/pingcap/ticdc/dm/pkg/streamer" "github.com/pingcap/ticdc/dm/pkg/terror" "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/relay/purger" + "github.com/pingcap/ticdc/dm/relay" ) // SourceWorker manages a source(upstream) which is mainly related to subtasks and relay. @@ -76,7 +76,7 @@ type SourceWorker struct { relayCancel context.CancelFunc relayWg sync.WaitGroup relayHolder RelayHolder - relayPurger purger.Purger + relayPurger relay.Purger taskStatusChecker TaskStatusChecker @@ -321,7 +321,7 @@ func (w *SourceWorker) EnableRelay() (err error) { // 2. initial relay holder, the cfg's password need decrypt w.relayHolder = NewRelayHolder(w.cfg) - relayPurger, err := w.relayHolder.Init(w.relayCtx, []purger.PurgeInterceptor{ + relayPurger, err := w.relayHolder.Init(w.relayCtx, []relay.PurgeInterceptor{ w, }) if err != nil { diff --git a/dm/dm/worker/source_worker_test.go b/dm/dm/worker/source_worker_test.go index 8fff70dfcca..7cf46630398 100644 --- a/dm/dm/worker/source_worker_test.go +++ b/dm/dm/worker/source_worker_test.go @@ -32,8 +32,8 @@ import ( "github.com/pingcap/ticdc/dm/pkg/conn" "github.com/pingcap/ticdc/dm/pkg/ha" "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/streamer" "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/ticdc/dm/relay" ) var emptyWorkerStatusInfoJSONLength = 25 @@ -242,7 +242,7 @@ var _ = Suite(&testWorkerFunctionalities{}) func (t *testWorkerFunctionalities) SetUpSuite(c *C) { NewRelayHolder = NewDummyRelayHolder NewSubTask = NewRealSubTask - createUnits = func(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, worker string, notifier streamer.EventNotifier) []unit.Unit { + createUnits = func(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, worker string, notifier relay.EventNotifier) []unit.Unit { atomic.AddInt32(&t.createUnitCount, 1) mockDumper := NewMockUnit(pb.UnitType_Dump) mockLoader := NewMockUnit(pb.UnitType_Load) @@ -417,7 +417,7 @@ func (t *testWorkerEtcdCompact) SetUpSuite(c *C) { cfg.UseRelay = false return NewRealSubTask(cfg, etcdClient, worker) } - createUnits = func(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, worker string, notifier streamer.EventNotifier) []unit.Unit { + createUnits = func(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, worker string, notifier relay.EventNotifier) []unit.Unit { mockDumper := NewMockUnit(pb.UnitType_Dump) mockLoader := NewMockUnit(pb.UnitType_Load) mockSync := NewMockUnit(pb.UnitType_Sync) diff --git a/dm/dm/worker/subtask.go b/dm/dm/worker/subtask.go index c8205cfcfc7..aa7cbeba221 100644 --- a/dm/dm/worker/subtask.go +++ b/dm/dm/worker/subtask.go @@ -34,9 +34,9 @@ import ( "github.com/pingcap/ticdc/dm/pkg/gtid" "github.com/pingcap/ticdc/dm/pkg/log" "github.com/pingcap/ticdc/dm/pkg/shardddl/pessimism" - "github.com/pingcap/ticdc/dm/pkg/streamer" "github.com/pingcap/ticdc/dm/pkg/terror" "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/ticdc/dm/relay" "github.com/pingcap/ticdc/dm/syncer" ) @@ -60,7 +60,7 @@ func (r relayNotifier) Notified() chan interface{} { var createUnits = createRealUnits // createRealUnits creates process units base on task mode. -func createRealUnits(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, workerName string, notifier streamer.EventNotifier) []unit.Unit { +func createRealUnits(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, workerName string, notifier relay.EventNotifier) []unit.Unit { failpoint.Inject("mockCreateUnitsDumpOnly", func(_ failpoint.Value) { log.L().Info("create mock worker units with dump unit only", zap.String("failpoint", "mockCreateUnitsDumpOnly")) failpoint.Return([]unit.Unit{dumpling.NewDumpling(cfg)}) @@ -120,7 +120,7 @@ type SubTask struct { workerName string - notifier streamer.EventNotifier + notifier relay.EventNotifier } // NewSubTask is subtask initializer diff --git a/dm/dm/worker/subtask_test.go b/dm/dm/worker/subtask_test.go index 06b77b2a9eb..111ab901c1f 100644 --- a/dm/dm/worker/subtask_test.go +++ b/dm/dm/worker/subtask_test.go @@ -24,8 +24,8 @@ import ( "github.com/pingcap/ticdc/dm/dumpling" "github.com/pingcap/ticdc/dm/loader" "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/streamer" "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/ticdc/dm/relay" "github.com/pingcap/ticdc/dm/syncer" . "github.com/pingcap/check" @@ -177,7 +177,7 @@ func (t *testSubTask) TestSubTaskNormalUsage(c *C) { defer func() { createUnits = createRealUnits }() - createUnits = func(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, worker string, notifier streamer.EventNotifier) []unit.Unit { + createUnits = func(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, worker string, notifier relay.EventNotifier) []unit.Unit { return nil } st.Run(pb.Stage_Running) @@ -186,7 +186,7 @@ func (t *testSubTask) TestSubTaskNormalUsage(c *C) { mockDumper := NewMockUnit(pb.UnitType_Dump) mockLoader := NewMockUnit(pb.UnitType_Load) - createUnits = func(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, worker string, notifier streamer.EventNotifier) []unit.Unit { + createUnits = func(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, worker string, notifier relay.EventNotifier) []unit.Unit { return []unit.Unit{mockDumper, mockLoader} } @@ -297,7 +297,7 @@ func (t *testSubTask) TestPauseAndResumeSubtask(c *C) { defer func() { createUnits = createRealUnits }() - createUnits = func(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, worker string, notifier streamer.EventNotifier) []unit.Unit { + createUnits = func(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, worker string, notifier relay.EventNotifier) []unit.Unit { return []unit.Unit{mockDumper, mockLoader} } @@ -421,7 +421,7 @@ func (t *testSubTask) TestSubtaskWithStage(c *C) { defer func() { createUnits = createRealUnits }() - createUnits = func(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, worker string, notifier streamer.EventNotifier) []unit.Unit { + createUnits = func(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, worker string, notifier relay.EventNotifier) []unit.Unit { return []unit.Unit{mockDumper, mockLoader} } @@ -446,7 +446,7 @@ func (t *testSubTask) TestSubtaskWithStage(c *C) { st = NewSubTaskWithStage(cfg, pb.Stage_Finished, nil, "worker") c.Assert(st.Stage(), DeepEquals, pb.Stage_Finished) - createUnits = func(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, worker string, notifier streamer.EventNotifier) []unit.Unit { + createUnits = func(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, worker string, notifier relay.EventNotifier) []unit.Unit { return []unit.Unit{mockDumper, mockLoader} } diff --git a/dm/pkg/binlog/writer/writer.go b/dm/pkg/binlog/writer/writer.go deleted file mode 100644 index bd9086fc0df..00000000000 --- a/dm/pkg/binlog/writer/writer.go +++ /dev/null @@ -1,32 +0,0 @@ -// Copyright 2019 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package writer - -// Writer is a binlog event writer, it may write binlog events to a binlog file, an in-memory structure or a TCP stream. -type Writer interface { - // Start prepares the writer for writing binlog events. - Start() error - - // Close closes the writer and release the resource. - Close() error - - // Write writes/appends a binlog event's rawData. - Write(rawData []byte) error - - // Flush flushes the buffered data to a stable storage or sends through the network. - Flush() error - - // Status returns the status of the writer. - Status() interface{} -} diff --git a/dm/pkg/streamer/util.go b/dm/pkg/streamer/util.go deleted file mode 100644 index bdd29073e19..00000000000 --- a/dm/pkg/streamer/util.go +++ /dev/null @@ -1,55 +0,0 @@ -// Copyright 2019 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package streamer - -import ( - "io" - "strings" - - "github.com/pingcap/errors" - - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" -) - -// getNextUUID gets (the nextUUID and its suffix) after the current UUID. -func getNextUUID(currUUID string, uuids []string) (string, string, error) { - for i := len(uuids) - 2; i >= 0; i-- { - if uuids[i] == currUUID { - nextUUID := uuids[i+1] - _, suffixInt, err := utils.ParseSuffixForUUID(nextUUID) - if err != nil { - return "", "", terror.Annotatef(err, "UUID %s", nextUUID) - } - return nextUUID, utils.SuffixIntToStr(suffixInt), nil - } - } - return "", "", nil -} - -// isIgnorableParseError checks whether is a ignorable error for `BinlogParser.ParseFile`. -func isIgnorableParseError(err error) bool { - if err == nil { - return false - } - - if strings.Contains(err.Error(), "err EOF") { - // NOTE: go-mysql returned err not includes caused err, but as message, ref: parser.go `parseSingleEvent` - return true - } else if errors.Cause(err) == io.EOF { - return true - } - - return false -} diff --git a/dm/pkg/streamer/util_test.go b/dm/pkg/streamer/util_test.go deleted file mode 100644 index f82cfe5a727..00000000000 --- a/dm/pkg/streamer/util_test.go +++ /dev/null @@ -1,124 +0,0 @@ -// Copyright 2019 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package streamer - -import ( - "io" - "testing" - - . "github.com/pingcap/check" - "github.com/pingcap/errors" -) - -var _ = Suite(&testUtilSuite{}) - -func TestSuite(t *testing.T) { - TestingT(t) -} - -type testUtilSuite struct{} - -func (t *testUtilSuite) TestGetNextUUID(c *C) { - UUIDs := []string{ - "b60868af-5a6f-11e9-9ea3-0242ac160006.000001", - "7acfedb5-3008-4fa2-9776-6bac42b025fe.000002", - "92ffd03b-813e-4391-b16a-177524e8d531.000003", - "338513ce-b24e-4ff8-9ded-9ac5aa8f4d74.000004", - } - cases := []struct { - currUUID string - UUIDs []string - nextUUID string - nextUUIDSuffix string - errMsgReg string - }{ - { - // empty current and UUID list - }, - { - // non-empty current UUID, but empty UUID list - currUUID: "b60868af-5a6f-11e9-9ea3-0242ac160006.000001", - }, - { - // empty current UUID, but non-empty UUID list - UUIDs: UUIDs, - }, - { - // current UUID in UUID list, has next UUID - currUUID: UUIDs[0], - UUIDs: UUIDs, - nextUUID: UUIDs[1], - nextUUIDSuffix: UUIDs[1][len(UUIDs[1])-6:], - }, - { - // current UUID in UUID list, but has no next UUID - currUUID: UUIDs[len(UUIDs)-1], - UUIDs: UUIDs, - }, - { - // current UUID not in UUID list - currUUID: "40ed16c1-f6f7-4012-aa9b-d360261d2b22.666666", - UUIDs: UUIDs, - }, - { - // invalid next UUID in UUID list - currUUID: UUIDs[len(UUIDs)-1], - UUIDs: append(UUIDs, "invalid-uuid"), - errMsgReg: ".*invalid-uuid.*", - }, - } - - for _, cs := range cases { - nu, nus, err := getNextUUID(cs.currUUID, cs.UUIDs) - if len(cs.errMsgReg) > 0 { - c.Assert(err, ErrorMatches, cs.errMsgReg) - } else { - c.Assert(err, IsNil) - } - c.Assert(nu, Equals, cs.nextUUID) - c.Assert(nus, Equals, cs.nextUUIDSuffix) - } -} - -func (t *testUtilSuite) TestIsIgnorableParseError(c *C) { - cases := []struct { - err error - ignorable bool - }{ - { - err: nil, - ignorable: false, - }, - { - err: io.EOF, - ignorable: true, - }, - { - err: errors.Annotate(io.EOF, "annotated end of file"), - ignorable: true, - }, - { - err: errors.New("get event header err EOF xxxx"), - ignorable: true, - }, - { - err: errors.New("some other error"), - ignorable: false, - }, - } - - for _, cs := range cases { - c.Assert(isIgnorableParseError(cs.err), Equals, cs.ignorable) - } -} diff --git a/dm/pkg/binlog/writer/file.go b/dm/relay/binlog_writer.go similarity index 80% rename from dm/pkg/binlog/writer/file.go rename to dm/relay/binlog_writer.go index 8295e89d223..2a98638532a 100644 --- a/dm/pkg/binlog/writer/file.go +++ b/dm/relay/binlog_writer.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package writer +package relay import ( "encoding/json" @@ -27,9 +27,9 @@ import ( "github.com/pingcap/ticdc/dm/pkg/terror" ) -// FileWriter is a binlog event writer which writes binlog events to a file. -type FileWriter struct { - cfg *FileWriterConfig +// BinlogWriter is a binlog event writer which writes binlog events to a file. +type BinlogWriter struct { + cfg *BinlogWriterConfig mu sync.RWMutex stage common.Stage @@ -40,15 +40,15 @@ type FileWriter struct { logger log.Logger } -// FileWriterStatus represents the status of a FileWriter. -type FileWriterStatus struct { +// BinlogWriterStatus represents the status of a BinlogWriter. +type BinlogWriterStatus struct { Stage string `json:"stage"` Filename string `json:"filename"` Offset int64 `json:"offset"` } // String implements Stringer.String. -func (s *FileWriterStatus) String() string { +func (s *BinlogWriterStatus) String() string { data, err := json.Marshal(s) if err != nil { // do not use %v/%+v for `s`, it will call this `String` recursively @@ -57,21 +57,21 @@ func (s *FileWriterStatus) String() string { return string(data) } -// FileWriterConfig is the configuration used by a FileWriter. -type FileWriterConfig struct { +// BinlogWriterConfig is the configuration used by a BinlogWriter. +type BinlogWriterConfig struct { Filename string } -// NewFileWriter creates a FileWriter instance. -func NewFileWriter(logger log.Logger, cfg *FileWriterConfig) Writer { - return &FileWriter{ +// NewBinlogWriter creates a BinlogWriter instance. +func NewBinlogWriter(logger log.Logger, cfg *BinlogWriterConfig) *BinlogWriter { + return &BinlogWriter{ cfg: cfg, logger: logger, } } // Start implements Writer.Start. -func (w *FileWriter) Start() error { +func (w *BinlogWriter) Start() error { w.mu.Lock() defer w.mu.Unlock() @@ -99,7 +99,7 @@ func (w *FileWriter) Start() error { } // Close implements Writer.Close. -func (w *FileWriter) Close() error { +func (w *BinlogWriter) Close() error { w.mu.Lock() defer w.mu.Unlock() @@ -122,7 +122,7 @@ func (w *FileWriter) Close() error { } // Write implements Writer.Write. -func (w *FileWriter) Write(rawData []byte) error { +func (w *BinlogWriter) Write(rawData []byte) error { w.mu.RLock() defer w.mu.RUnlock() @@ -137,7 +137,7 @@ func (w *FileWriter) Write(rawData []byte) error { } // Flush implements Writer.Flush. -func (w *FileWriter) Flush() error { +func (w *BinlogWriter) Flush() error { w.mu.RLock() defer w.mu.RUnlock() @@ -149,12 +149,12 @@ func (w *FileWriter) Flush() error { } // Status implements Writer.Status. -func (w *FileWriter) Status() interface{} { +func (w *BinlogWriter) Status() interface{} { w.mu.RLock() stage := w.stage w.mu.RUnlock() - return &FileWriterStatus{ + return &BinlogWriterStatus{ Stage: stage.String(), Filename: w.cfg.Filename, Offset: w.offset.Load(), @@ -162,7 +162,7 @@ func (w *FileWriter) Status() interface{} { } // flush flushes the buffered data to the disk. -func (w *FileWriter) flush() error { +func (w *BinlogWriter) flush() error { if w.file == nil { return terror.ErrBinlogWriterFileNotOpened.Generate(w.cfg.Filename) } diff --git a/dm/pkg/binlog/writer/file_test.go b/dm/relay/binlog_writer_test.go similarity index 87% rename from dm/pkg/binlog/writer/file_test.go rename to dm/relay/binlog_writer_test.go index 3b8ba49fb23..2d6bb0cbfd7 100644 --- a/dm/pkg/binlog/writer/file_test.go +++ b/dm/relay/binlog_writer_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package writer +package relay import ( "bytes" @@ -27,30 +27,30 @@ import ( "github.com/pingcap/ticdc/dm/pkg/log" ) -func TestSuite(t *testing.T) { +func TestBinlogWriterSuite(t *testing.T) { TestingT(t) } -var _ = Suite(&testFileWriterSuite{}) +var _ = Suite(&testBinlogWriterSuite{}) -type testFileWriterSuite struct{} +type testBinlogWriterSuite struct{} -func (t *testFileWriterSuite) TestWrite(c *C) { +func (t *testBinlogWriterSuite) TestWrite(c *C) { dir := c.MkDir() filename := filepath.Join(dir, "test-mysql-bin.000001") var ( - cfg = &FileWriterConfig{ + cfg = &BinlogWriterConfig{ Filename: filename, } allData bytes.Buffer ) - w := NewFileWriter(log.L(), cfg) + w := NewBinlogWriter(log.L(), cfg) c.Assert(w, NotNil) // check status, stageNew status := w.Status() - fwStatus, ok := status.(*FileWriterStatus) + fwStatus, ok := status.(*BinlogWriterStatus) c.Assert(ok, IsTrue) c.Assert(fwStatus.Stage, Equals, common.StageNew.String()) c.Assert(fwStatus.Filename, Equals, filename) @@ -71,7 +71,7 @@ func (t *testFileWriterSuite) TestWrite(c *C) { // check status, stagePrepared status = w.Status() - fwStatus, ok = status.(*FileWriterStatus) + fwStatus, ok = status.(*BinlogWriterStatus) c.Assert(ok, IsTrue) c.Assert(fwStatus.Stage, Equals, common.StagePrepared.String()) c.Assert(fwStatus.Filename, Equals, filename) @@ -101,7 +101,7 @@ func (t *testFileWriterSuite) TestWrite(c *C) { // check status, stageClosed status = w.Status() - fwStatus, ok = status.(*FileWriterStatus) + fwStatus, ok = status.(*BinlogWriterStatus) c.Assert(ok, IsTrue) c.Assert(fwStatus.Stage, Equals, common.StageClosed.String()) c.Assert(fwStatus.Filename, Equals, filename) diff --git a/dm/relay/config.go b/dm/relay/config.go index 5efac22e0ad..aa75d8ef527 100644 --- a/dm/relay/config.go +++ b/dm/relay/config.go @@ -18,7 +18,6 @@ import ( "github.com/pingcap/ticdc/dm/dm/config" "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/relay/retry" ) // Config is the configuration for Relay. @@ -38,7 +37,7 @@ type Config struct { UUIDSuffix int `toml:"-" json:"-"` // for binlog reader retry - ReaderRetry retry.ReaderRetryConfig `toml:"reader-retry" json:"reader-retry"` + ReaderRetry ReaderRetryConfig `toml:"reader-retry" json:"reader-retry"` } func (c *Config) String() string { @@ -63,7 +62,7 @@ func FromSourceCfg(sourceCfg *config.SourceConfig) *Config { BinLogName: clone.RelayBinLogName, BinlogGTID: clone.RelayBinlogGTID, UUIDSuffix: clone.UUIDSuffix, - ReaderRetry: retry.ReaderRetryConfig{ // we use config from TaskChecker now + ReaderRetry: ReaderRetryConfig{ // we use config from TaskChecker now BackoffRollback: clone.Checker.BackoffRollback.Duration, BackoffMax: clone.Checker.BackoffMax.Duration, BackoffMin: clone.Checker.BackoffMin.Duration, diff --git a/dm/relay/reader/error.go b/dm/relay/error.go similarity index 98% rename from dm/relay/reader/error.go rename to dm/relay/error.go index 0587a30a961..e5284e33c44 100644 --- a/dm/relay/reader/error.go +++ b/dm/relay/error.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package reader +package relay import ( "context" diff --git a/dm/relay/reader/error_test.go b/dm/relay/error_test.go similarity index 98% rename from dm/relay/reader/error_test.go rename to dm/relay/error_test.go index 38200a22bfc..e2ab04d2046 100644 --- a/dm/relay/reader/error_test.go +++ b/dm/relay/error_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package reader +package relay import ( "context" diff --git a/dm/pkg/streamer/file.go b/dm/relay/file.go similarity index 99% rename from dm/pkg/streamer/file.go rename to dm/relay/file.go index 8a426075efd..0317b2f2fac 100644 --- a/dm/pkg/streamer/file.go +++ b/dm/relay/file.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package streamer +package relay import ( "context" @@ -219,7 +219,7 @@ func (r *relayLogFileChecker) relayLogUpdatedOrNewCreated(ctx context.Context, u // binlog file may have rotated if we read nothing last time(either it's the first read or after notified) lastReadCnt := r.endOffset - r.beginOffset if lastReadCnt == 0 { - meta := &Meta{} + meta := &LocalMeta{} _, err := toml.DecodeFile(filepath.Join(r.latestRelayLogDir, utils.MetaFilename), meta) if err != nil { errCh <- terror.Annotate(err, "decode relay meta toml file failed") diff --git a/dm/pkg/streamer/file_test.go b/dm/relay/file_test.go similarity index 99% rename from dm/pkg/streamer/file_test.go rename to dm/relay/file_test.go index f02a89043ae..22c5abd6acf 100644 --- a/dm/pkg/streamer/file_test.go +++ b/dm/relay/file_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package streamer +package relay import ( "bytes" @@ -358,10 +358,10 @@ func (t *testFileSuite) TestrelayLogUpdatedOrNewCreated(c *C) { } rotateRelayFile := func(filename string) { - meta := Meta{BinLogName: filename, BinLogPos: binlogPos, BinlogGTID: binlogGTID} + meta := LocalMeta{BinLogName: filename, BinLogPos: binlogPos, BinlogGTID: binlogGTID} metaFile, err2 := os.Create(path.Join(subDir, utils.MetaFilename)) c.Assert(err2, IsNil) - err = toml.NewEncoder(metaFile).Encode(meta) + err = toml.NewEncoder(metaFile).Encode(&meta) c.Assert(err, IsNil) _ = metaFile.Close() } diff --git a/dm/relay/writer/file_util.go b/dm/relay/file_util.go similarity index 99% rename from dm/relay/writer/file_util.go rename to dm/relay/file_util.go index c8cf5a01d2d..a1190b99d30 100644 --- a/dm/relay/writer/file_util.go +++ b/dm/relay/file_util.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package writer +package relay import ( "bytes" diff --git a/dm/relay/writer/file_util_test.go b/dm/relay/file_util_test.go similarity index 90% rename from dm/relay/writer/file_util_test.go rename to dm/relay/file_util_test.go index 7b1d158a827..8f7584dbb4d 100644 --- a/dm/relay/writer/file_util_test.go +++ b/dm/relay/file_util_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package writer +package relay import ( "bytes" @@ -402,72 +402,6 @@ func (t *testFileUtilSuite) testGetTxnPosGTIDs(c *check.C, filename, flavor, pre c.Assert(gSet, check.DeepEquals, expectedGTIDs) } -// genBinlogEventsWithGTIDs generates some binlog events used by testFileUtilSuite and testFileWriterSuite. -// now, its generated events including 3 DDL and 10 DML. -// nolint:unparam -func genBinlogEventsWithGTIDs(c *check.C, flavor string, previousGTIDSet, latestGTID1, latestGTID2 gtid.Set) (*event.Generator, []*replication.BinlogEvent, []byte) { - var ( - serverID uint32 = 11 - latestPos uint32 - latestXID uint64 = 10 - - allEvents = make([]*replication.BinlogEvent, 0, 50) - allData bytes.Buffer - ) - - // use a binlog event generator to generate some binlog events. - g, err := event.NewGenerator(flavor, serverID, latestPos, latestGTID1, previousGTIDSet, latestXID) - c.Assert(err, check.IsNil) - - // file header with FormatDescriptionEvent and PreviousGTIDsEvent - events, data, err := g.GenFileHeader() - c.Assert(err, check.IsNil) - allEvents = append(allEvents, events...) - allData.Write(data) - - // CREATE DATABASE/TABLE, 3 DDL - queries := []string{ - "CREATE DATABASE `db`", - "CREATE TABLE `db`.`tbl1` (c1 INT)", - "CREATE TABLE `db`.`tbl2` (c1 INT)", - } - for _, query := range queries { - events, data, err = g.GenDDLEvents("db", query) - c.Assert(err, check.IsNil) - allEvents = append(allEvents, events...) - allData.Write(data) - } - - // DMLs, 10 DML - g.LatestGTID = latestGTID2 // use another latest GTID with different SID/DomainID - var ( - tableID uint64 = 8 - columnType = []byte{gmysql.MYSQL_TYPE_LONG} - eventType = replication.WRITE_ROWS_EVENTv2 - schema = "db" - table = "tbl1" - ) - for i := 0; i < 10; i++ { - insertRows := make([][]interface{}, 0, 1) - insertRows = append(insertRows, []interface{}{int32(i)}) - dmlData := []*event.DMLData{ - { - TableID: tableID, - Schema: schema, - Table: table, - ColumnType: columnType, - Rows: insertRows, - }, - } - events, data, err = g.GenDMLEvents(eventType, dmlData) - c.Assert(err, check.IsNil) - allEvents = append(allEvents, events...) - allData.Write(data) - } - - return g, allEvents, allData.Bytes() -} - func (t *testFileUtilSuite) TestGetTxnPosGTIDsNoGTID(c *check.C) { // generate some events but without GTID enabled var ( diff --git a/dm/pkg/streamer/reader.go b/dm/relay/local_reader.go similarity index 97% rename from dm/pkg/streamer/reader.go rename to dm/relay/local_reader.go index 4d01ea0cd8c..ad05656e4b5 100644 --- a/dm/pkg/streamer/reader.go +++ b/dm/relay/local_reader.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package streamer +package relay import ( "context" @@ -43,14 +43,6 @@ import ( // this is mainly happened when upstream master changed when relay log not finish reading a transaction. var ErrorMaybeDuplicateEvent = errors.New("truncate binlog file found, event may be duplicated") -// Meta represents binlog meta information in relay.meta. -type Meta struct { - BinLogName string `toml:"binlog-name" json:"binlog-name"` - BinLogPos uint32 `toml:"binlog-pos" json:"binlog-pos"` - BinlogGTID string `toml:"binlog-gtid" json:"binlog-gtid"` - UUID string `toml:"-" json:"-"` -} - // polling interval for watcher. var watcherInterval = 100 * time.Millisecond @@ -215,7 +207,7 @@ func (r *BinlogReader) getPosByGTID(gset mysql.GTIDSet) (*mysql.Position, error) // StartSyncByPos start sync by pos // TODO: thread-safe? -func (r *BinlogReader) StartSyncByPos(pos mysql.Position) (Streamer, error) { +func (r *BinlogReader) StartSyncByPos(pos mysql.Position) (reader.Streamer, error) { if pos.Name == "" { return nil, terror.ErrBinlogFileNotSpecified.Generate() } @@ -255,7 +247,7 @@ func (r *BinlogReader) StartSyncByPos(pos mysql.Position) (Streamer, error) { } // StartSyncByGTID start sync by gtid. -func (r *BinlogReader) StartSyncByGTID(gset mysql.GTIDSet) (Streamer, error) { +func (r *BinlogReader) StartSyncByGTID(gset mysql.GTIDSet) (reader.Streamer, error) { r.tctx.L().Info("begin to sync binlog", zap.Stringer("GTID Set", gset)) r.usingGTID = true diff --git a/dm/pkg/streamer/reader_test.go b/dm/relay/local_reader_test.go similarity index 98% rename from dm/pkg/streamer/reader_test.go rename to dm/relay/local_reader_test.go index 5e7942ac509..f77d96b02b4 100644 --- a/dm/pkg/streamer/reader_test.go +++ b/dm/relay/local_reader_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package streamer +package relay import ( "bytes" @@ -37,6 +37,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/ticdc/dm/pkg/binlog/event" + "github.com/pingcap/ticdc/dm/pkg/binlog/reader" "github.com/pingcap/ticdc/dm/pkg/gtid" "github.com/pingcap/ticdc/dm/pkg/log" "github.com/pingcap/ticdc/dm/pkg/terror" @@ -61,11 +62,11 @@ func (t *testReaderSuite) SetUpSuite(c *C) { t.lastPos = 0 t.lastGTID, err = gtid.ParserGTID(gmysql.MySQLFlavor, "ba8f633f-1f15-11eb-b1c7-0242ac110002:0") c.Assert(err, IsNil) - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/pkg/streamer/SetHeartbeatInterval", "return(10000)"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/relay/SetHeartbeatInterval", "return(10000)"), IsNil) } func (t *testReaderSuite) TearDownSuite(c *C) { - c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/pkg/streamer/SetHeartbeatInterval"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/relay/SetHeartbeatInterval"), IsNil) } func (t *testReaderSuite) TestParseFileBase(c *C) { @@ -542,7 +543,7 @@ func (t *testReaderSuite) TestStartSyncByPos(c *C) { r.Close() } -func readNEvents(ctx context.Context, c *C, s Streamer, l int) []*replication.BinlogEvent { +func readNEvents(ctx context.Context, c *C, s reader.Streamer, l int) []*replication.BinlogEvent { var result []*replication.BinlogEvent for { ev, err2 := s.GetEvent(ctx) @@ -957,10 +958,10 @@ func (t *testReaderSuite) TestReParseUsingGTID(c *C) { _, err = f.Write(replication.BinLogFileHeader) c.Assert(err, IsNil) - meta := Meta{BinLogName: file, BinLogPos: latestPos, BinlogGTID: startGTID.String()} + meta := LocalMeta{BinLogName: file, BinLogPos: latestPos, BinlogGTID: startGTID.String()} metaFile, err := os.Create(path.Join(uuidDir, utils.MetaFilename)) c.Assert(err, IsNil) - c.Assert(toml.NewEncoder(metaFile).Encode(meta), IsNil) + c.Assert(toml.NewEncoder(metaFile).Encode(&meta), IsNil) c.Assert(metaFile.Close(), IsNil) // prepare some regular events, @@ -1125,7 +1126,7 @@ func (t *testReaderSuite) genEvents(c *C, eventTypes []replication.EventType, la return events, latestPos, latestGTID, pGset } -func (t *testReaderSuite) purgeStreamer(c *C, s Streamer) { +func (t *testReaderSuite) purgeStreamer(c *C, s reader.Streamer) { ctx, cancel := context.WithTimeout(context.Background(), 10*time.Millisecond) defer cancel() @@ -1142,7 +1143,7 @@ func (t *testReaderSuite) purgeStreamer(c *C, s Streamer) { } } -func (t *testReaderSuite) verifyNoEventsInStreamer(c *C, s Streamer) { +func (t *testReaderSuite) verifyNoEventsInStreamer(c *C, s reader.Streamer) { ctx, cancel := context.WithTimeout(context.Background(), 10*time.Millisecond) defer cancel() @@ -1181,10 +1182,10 @@ func (t *testReaderSuite) writeUUIDs(c *C, relayDir string, uuids []string) []by } func (t *testReaderSuite) createMetaFile(c *C, relayDirPath, binlogFileName string, pos uint32, gtid string) { - meta := Meta{BinLogName: binlogFileName, BinLogPos: pos, BinlogGTID: gtid} + meta := LocalMeta{BinLogName: binlogFileName, BinLogPos: pos, BinlogGTID: gtid} metaFile, err2 := os.Create(path.Join(relayDirPath, utils.MetaFilename)) c.Assert(err2, IsNil) - err := toml.NewEncoder(metaFile).Encode(meta) + err := toml.NewEncoder(metaFile).Encode(&meta) c.Assert(err, IsNil) metaFile.Close() } diff --git a/dm/relay/purge_strategy.go b/dm/relay/purge_strategy.go new file mode 100644 index 00000000000..0fa535dcf57 --- /dev/null +++ b/dm/relay/purge_strategy.go @@ -0,0 +1,360 @@ +// Copyright 2019 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package relay + +import ( + "fmt" + "strings" + "time" + + "go.uber.org/atomic" + "go.uber.org/zap" + + "github.com/pingcap/ticdc/dm/pkg/log" + "github.com/pingcap/ticdc/dm/pkg/streamer" + "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/ticdc/dm/pkg/utils" +) + +type strategyType uint32 + +const ( + strategyNone strategyType = iota + strategyInactive + strategyFilename + strategyTime + strategySpace +) + +func (s strategyType) String() string { + switch s { + case strategyInactive: + return "inactive strategy" + case strategyFilename: + return "filename strategy" + case strategyTime: + return "time strategy" + case strategySpace: + return "space strategy" + default: + return "unknown strategy" + } +} + +// PurgeStrategy represents a relay log purge strategy +// two purge behaviors +// 1. purge in the background +// 2. do one time purge process +// a strategy can support both or one of them. +type PurgeStrategy interface { + // Check checks whether need to do the purge in the background automatically + Check(args interface{}) (bool, error) + + // Do does the purge process one time + Do(args interface{}) error + + // Purging indicates whether is doing purge + Purging() bool + + // Type returns the strategy type + Type() strategyType +} + +// StrategyArgs represents args needed by purge strategy. +type StrategyArgs interface { + // SetActiveRelayLog sets active relay log info in args + // this should be called before do the purging + SetActiveRelayLog(active *streamer.RelayLogInfo) +} + +var fakeStrategyTaskName = strategyFilename.String() + +// filenameArgs represents args needed by filenameStrategy +// NOTE: should handle master-slave switch. +type filenameArgs struct { + relayBaseDir string + filename string // specified end safe filename + subDir string // sub dir for @filename, empty indicates latest sub dir + uuids []string + safeRelayLog *streamer.RelayLogInfo // all relay log files prior to this should be purged +} + +func (fa *filenameArgs) SetActiveRelayLog(active *streamer.RelayLogInfo) { + uuid := fa.subDir + if len(uuid) == 0 && len(fa.uuids) > 0 { + // no sub dir specified, use the latest one + uuid = fa.uuids[len(fa.uuids)-1] + } + _, endSuffix, _ := utils.ParseSuffixForUUID(uuid) + + safeRelayLog := &streamer.RelayLogInfo{ + TaskName: fakeStrategyTaskName, + UUID: uuid, + UUIDSuffix: endSuffix, + Filename: fa.filename, + } + + if active.Earlier(safeRelayLog) { + safeRelayLog = active + } + + fa.safeRelayLog = safeRelayLog + + // discard newer UUIDs + uuids := make([]string, 0, len(fa.uuids)) + for _, uuid := range fa.uuids { + _, suffix, _ := utils.ParseSuffixForUUID(uuid) + if suffix > endSuffix { + break + } + uuids = append(uuids, uuid) + } + fa.uuids = uuids +} + +func (fa *filenameArgs) String() string { + return fmt.Sprintf("(RelayBaseDir: %s, Filename: %s, SubDir: %s, UUIDs: %s, SafeRelayLog: %s)", + fa.relayBaseDir, fa.filename, fa.subDir, strings.Join(fa.uuids, ";"), fa.safeRelayLog) +} + +// filenameStrategy represents a relay purge strategy by filename +// similar to `PURGE BINARY LOGS TO`. +type filenameStrategy struct { + purging atomic.Bool + + logger log.Logger +} + +func newFilenameStrategy() PurgeStrategy { + return &filenameStrategy{ + logger: log.With(zap.String("component", "relay purger"), zap.String("strategy", "file name")), + } +} + +func (s *filenameStrategy) Check(args interface{}) (bool, error) { + // do not support purge in the background + return false, nil +} + +func (s *filenameStrategy) Do(args interface{}) error { + if !s.purging.CAS(false, true) { + return terror.ErrRelayThisStrategyIsPurging.Generate() + } + defer s.purging.Store(false) + + fa, ok := args.(*filenameArgs) + if !ok { + return terror.ErrRelayPurgeArgsNotValid.Generate(args, args) + } + + return purgeRelayFilesBeforeFile(s.logger, fa.relayBaseDir, fa.uuids, fa.safeRelayLog) +} + +func (s *filenameStrategy) Purging() bool { + return s.purging.Load() +} + +func (s *filenameStrategy) Type() strategyType { + return strategyFilename +} + +// inactiveArgs represents args needed by inactiveStrategy. +type inactiveArgs struct { + relayBaseDir string + uuids []string + activeRelayLog *streamer.RelayLogInfo // earliest active relay log info +} + +func (ia *inactiveArgs) SetActiveRelayLog(active *streamer.RelayLogInfo) { + ia.activeRelayLog = active +} + +func (ia *inactiveArgs) String() string { + return fmt.Sprintf("(RelayBaseDir: %s, UUIDs: %s, ActiveRelayLog: %s)", + ia.relayBaseDir, strings.Join(ia.uuids, ";"), ia.activeRelayLog) +} + +// inactiveStrategy represents a relay purge strategy which purge all inactive relay log files +// definition of inactive relay log files: +// * not writing by relay unit +// * not reading by sync unit and will not be read by any running tasks +// TODO zxc: judge tasks are running dumper / loader +type inactiveStrategy struct { + purging atomic.Bool + + logger log.Logger +} + +func newInactiveStrategy() PurgeStrategy { + return &inactiveStrategy{ + logger: log.With(zap.String("component", "relay purger"), zap.String("strategy", "inactive binlog file")), + } +} + +func (s *inactiveStrategy) Check(args interface{}) (bool, error) { + // do not support purge in the background + return false, nil +} + +func (s *inactiveStrategy) Do(args interface{}) error { + if !s.purging.CAS(false, true) { + return terror.ErrRelayThisStrategyIsPurging.Generate() + } + defer s.purging.Store(false) + + ia, ok := args.(*inactiveArgs) + if !ok { + return terror.ErrRelayPurgeArgsNotValid.Generate(args, args) + } + + return purgeRelayFilesBeforeFile(s.logger, ia.relayBaseDir, ia.uuids, ia.activeRelayLog) +} + +func (s *inactiveStrategy) Purging() bool { + return s.purging.Load() +} + +func (s *inactiveStrategy) Type() strategyType { + return strategyInactive +} + +// spaceArgs represents args needed by spaceStrategy. +type spaceArgs struct { + relayBaseDir string + remainSpace int64 // if remain space (GB) in @RelayBaseDir less than this, then it can be purged + uuids []string + activeRelayLog *streamer.RelayLogInfo // earliest active relay log info +} + +func (sa *spaceArgs) SetActiveRelayLog(active *streamer.RelayLogInfo) { + sa.activeRelayLog = active +} + +func (sa *spaceArgs) String() string { + return fmt.Sprintf("(RelayBaseDir: %s, AllowMinRemainSpace: %dGB, UUIDs: %s, ActiveRelayLog: %s)", + sa.relayBaseDir, sa.remainSpace, strings.Join(sa.uuids, ";"), sa.activeRelayLog) +} + +// spaceStrategy represents a relay purge strategy by remain space in dm-worker node. +type spaceStrategy struct { + purging atomic.Bool + + logger log.Logger +} + +func newSpaceStrategy() PurgeStrategy { + return &spaceStrategy{ + logger: log.With(zap.String("component", "relay purger"), zap.String("strategy", "space")), + } +} + +func (s *spaceStrategy) Check(args interface{}) (bool, error) { + sa, ok := args.(*spaceArgs) + if !ok { + return false, terror.ErrRelayPurgeArgsNotValid.Generate(args, args) + } + + storageSize, err := utils.GetStorageSize(sa.relayBaseDir) + if err != nil { + return false, terror.Annotatef(err, "get storage size for directory %s", sa.relayBaseDir) + } + + requiredBytes := uint64(sa.remainSpace) * 1024 * 1024 * 1024 + return storageSize.Available < requiredBytes, nil +} + +func (s *spaceStrategy) Do(args interface{}) error { + if !s.purging.CAS(false, true) { + return terror.ErrRelayThisStrategyIsPurging.Generate() + } + defer s.purging.Store(false) + + sa, ok := args.(*spaceArgs) + if !ok { + return terror.ErrRelayPurgeArgsNotValid.Generate(args, args) + } + + // NOTE: we purge all inactive relay log files when available space less than @remainSpace + // maybe we can refine this to purge only part of this files every time + return purgeRelayFilesBeforeFile(s.logger, sa.relayBaseDir, sa.uuids, sa.activeRelayLog) +} + +func (s *spaceStrategy) Purging() bool { + return s.purging.Load() +} + +func (s *spaceStrategy) Type() strategyType { + return strategySpace +} + +// timeArgs represents args needed by timeStrategy. +type timeArgs struct { + relayBaseDir string + safeTime time.Time // if file's modified time is older than this, then it can be purged + uuids []string + activeRelayLog *streamer.RelayLogInfo // earliest active relay log info +} + +func (ta *timeArgs) SetActiveRelayLog(active *streamer.RelayLogInfo) { + ta.activeRelayLog = active +} + +func (ta *timeArgs) String() string { + return fmt.Sprintf("(RelayBaseDir: %s, SafeTime: %s, UUIDs: %s, ActiveRelayLog: %s)", + ta.relayBaseDir, ta.safeTime, strings.Join(ta.uuids, ";"), ta.activeRelayLog) +} + +// timeStrategy represents a relay purge strategy by time +// similar to `PURGE BINARY LOGS BEFORE` in MySQL. +type timeStrategy struct { + purging atomic.Bool + + logger log.Logger +} + +func newTimeStrategy() PurgeStrategy { + return &timeStrategy{ + logger: log.With(zap.String("component", "relay purger"), zap.String("strategy", "time")), + } +} + +func (s *timeStrategy) Check(args interface{}) (bool, error) { + // for time strategy, we always try to do the purging + return true, nil +} + +func (s *timeStrategy) Stop() { +} + +func (s *timeStrategy) Do(args interface{}) error { + if !s.purging.CAS(false, true) { + return terror.ErrRelayThisStrategyIsPurging.Generate() + } + defer s.purging.Store(false) + + ta, ok := args.(*timeArgs) + if !ok { + return terror.ErrRelayPurgeArgsNotValid.Generate(args, args) + } + + return purgeRelayFilesBeforeFileAndTime(s.logger, ta.relayBaseDir, ta.uuids, ta.activeRelayLog, ta.safeTime) +} + +func (s *timeStrategy) Purging() bool { + return s.purging.Load() +} + +func (s *timeStrategy) Type() strategyType { + return strategyTime +} diff --git a/dm/relay/purger/purger.go b/dm/relay/purger.go similarity index 90% rename from dm/relay/purger/purger.go rename to dm/relay/purger.go index c869748536b..95e46e0eca7 100644 --- a/dm/relay/purger/purger.go +++ b/dm/relay/purger.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package purger +package relay import ( "context" @@ -30,8 +30,8 @@ import ( "github.com/pingcap/ticdc/dm/pkg/utils" ) -// RelayOperator represents an operator for relay log files, like writer, reader. -type RelayOperator interface { +// Operator represents an operator for relay log files, like writer, reader. +type Operator interface { // EarliestActiveRelayLog returns the earliest active relay log info in this operator EarliestActiveRelayLog() *streamer.RelayLogInfo } @@ -63,8 +63,8 @@ type Purger interface { // NewPurger creates a new purger. var NewPurger = NewRelayPurger -// RelayPurger purges relay log according to some strategies. -type RelayPurger struct { +// relayPurger purges relay log according to some strategies. +type relayPurger struct { lock sync.RWMutex wg sync.WaitGroup cancel context.CancelFunc @@ -74,7 +74,7 @@ type RelayPurger struct { cfg config.PurgeConfig baseRelayDir string indexPath string // server-uuid.index file path - operators []RelayOperator + operators []Operator interceptors []PurgeInterceptor strategies map[strategyType]PurgeStrategy @@ -82,8 +82,8 @@ type RelayPurger struct { } // NewRelayPurger creates a new purger. -func NewRelayPurger(cfg config.PurgeConfig, baseRelayDir string, operators []RelayOperator, interceptors []PurgeInterceptor) Purger { - p := &RelayPurger{ +func NewRelayPurger(cfg config.PurgeConfig, baseRelayDir string, operators []Operator, interceptors []PurgeInterceptor) Purger { + p := &relayPurger{ cfg: cfg, baseRelayDir: baseRelayDir, indexPath: filepath.Join(baseRelayDir, utils.UUIDIndexFilename), @@ -103,7 +103,7 @@ func NewRelayPurger(cfg config.PurgeConfig, baseRelayDir string, operators []Rel } // Start starts strategies by config. -func (p *RelayPurger) Start() { +func (p *relayPurger) Start() { if !p.running.CAS(stageNew, stageRunning) { return } @@ -124,7 +124,7 @@ func (p *RelayPurger) Start() { // run starts running the process // NOTE: ensure run is called at most once of a Purger. -func (p *RelayPurger) run() { +func (p *relayPurger) run() { ticker := time.NewTicker(time.Duration(p.cfg.Interval) * time.Second) defer ticker.Stop() @@ -143,7 +143,7 @@ func (p *RelayPurger) run() { } // Close stops the started strategies. -func (p *RelayPurger) Close() { +func (p *relayPurger) Close() { if !p.running.CAS(stageRunning, stageClosed) { return } @@ -159,12 +159,12 @@ func (p *RelayPurger) Close() { } // Purging returns whether the purger is purging. -func (p *RelayPurger) Purging() bool { +func (p *relayPurger) Purging() bool { return p.purgingStrategy.Load() != uint32(strategyNone) } // Do does the purge process one time. -func (p *RelayPurger) Do(ctx context.Context, req *pb.PurgeRelayRequest) error { +func (p *relayPurger) Do(ctx context.Context, req *pb.PurgeRelayRequest) error { uuids, err := utils.ParseUUIDIndex(p.indexPath) if err != nil { return terror.Annotatef(err, "parse UUID index file %s", p.indexPath) @@ -201,7 +201,7 @@ func (p *RelayPurger) Do(ctx context.Context, req *pb.PurgeRelayRequest) error { } // tryPurge tries to do purge by check condition first. -func (p *RelayPurger) tryPurge() { +func (p *relayPurger) tryPurge() { strategy, args, err := p.check() if err != nil { p.logger.Error("check whether need to purge relay log files in background", zap.Error(err)) @@ -217,7 +217,7 @@ func (p *RelayPurger) tryPurge() { } // doPurge does the purging operation. -func (p *RelayPurger) doPurge(ps PurgeStrategy, args StrategyArgs) error { +func (p *relayPurger) doPurge(ps PurgeStrategy, args StrategyArgs) error { if !p.purgingStrategy.CAS(uint32(strategyNone), uint32(ps.Type())) { return terror.ErrRelayOtherStrategyIsPurging.Generate(ps.Type()) } @@ -241,7 +241,7 @@ func (p *RelayPurger) doPurge(ps PurgeStrategy, args StrategyArgs) error { return ps.Do(args) } -func (p *RelayPurger) check() (PurgeStrategy, StrategyArgs, error) { +func (p *relayPurger) check() (PurgeStrategy, StrategyArgs, error) { p.logger.Info("checking whether needing to purge relay log files") uuids, err := utils.ParseUUIDIndex(p.indexPath) @@ -292,7 +292,7 @@ func (p *RelayPurger) check() (PurgeStrategy, StrategyArgs, error) { } // earliestActiveRelayLog returns the current earliest active relay log info. -func (p *RelayPurger) earliestActiveRelayLog() *streamer.RelayLogInfo { +func (p *relayPurger) earliestActiveRelayLog() *streamer.RelayLogInfo { var earliest *streamer.RelayLogInfo for _, op := range p.operators { info := op.EarliestActiveRelayLog() @@ -309,7 +309,7 @@ func (p *RelayPurger) earliestActiveRelayLog() *streamer.RelayLogInfo { type dummyPurger struct{} // NewDummyPurger returns a dummy purger. -func NewDummyPurger(cfg config.PurgeConfig, baseRelayDir string, operators []RelayOperator, interceptors []PurgeInterceptor) Purger { +func NewDummyPurger(cfg config.PurgeConfig, baseRelayDir string, operators []Operator, interceptors []PurgeInterceptor) Purger { return &dummyPurger{} } diff --git a/dm/relay/purger/strategy.go b/dm/relay/purger/strategy.go deleted file mode 100644 index 2ee1e5955f8..00000000000 --- a/dm/relay/purger/strategy.go +++ /dev/null @@ -1,67 +0,0 @@ -// Copyright 2019 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package purger - -import "github.com/pingcap/ticdc/dm/pkg/streamer" - -type strategyType uint32 - -const ( - strategyNone strategyType = iota - strategyInactive - strategyFilename - strategyTime - strategySpace -) - -func (s strategyType) String() string { - switch s { - case strategyInactive: - return "inactive strategy" - case strategyFilename: - return "filename strategy" - case strategyTime: - return "time strategy" - case strategySpace: - return "space strategy" - default: - return "unknown strategy" - } -} - -// PurgeStrategy represents a relay log purge strategy -// two purge behaviors -// 1. purge in the background -// 2. do one time purge process -// a strategy can support both or one of them. -type PurgeStrategy interface { - // Check checks whether need to do the purge in the background automatically - Check(args interface{}) (bool, error) - - // Do does the purge process one time - Do(args interface{}) error - - // Purging indicates whether is doing purge - Purging() bool - - // Type returns the strategy type - Type() strategyType -} - -// StrategyArgs represents args needed by purge strategy. -type StrategyArgs interface { - // SetActiveRelayLog sets active relay log info in args - // this should be called before do the purging - SetActiveRelayLog(active *streamer.RelayLogInfo) -} diff --git a/dm/relay/purger/strategy_filename.go b/dm/relay/purger/strategy_filename.go deleted file mode 100644 index 6d346b1139d..00000000000 --- a/dm/relay/purger/strategy_filename.go +++ /dev/null @@ -1,118 +0,0 @@ -// Copyright 2019 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package purger - -import ( - "fmt" - "strings" - - "go.uber.org/atomic" - "go.uber.org/zap" - - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/streamer" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" -) - -var fakeTaskName = strategyFilename.String() - -// filenameArgs represents args needed by filenameStrategy -// NOTE: should handle master-slave switch. -type filenameArgs struct { - relayBaseDir string - filename string // specified end safe filename - subDir string // sub dir for @filename, empty indicates latest sub dir - uuids []string - safeRelayLog *streamer.RelayLogInfo // all relay log files prior to this should be purged -} - -func (fa *filenameArgs) SetActiveRelayLog(active *streamer.RelayLogInfo) { - uuid := fa.subDir - if len(uuid) == 0 && len(fa.uuids) > 0 { - // no sub dir specified, use the latest one - uuid = fa.uuids[len(fa.uuids)-1] - } - _, endSuffix, _ := utils.ParseSuffixForUUID(uuid) - - safeRelayLog := &streamer.RelayLogInfo{ - TaskName: fakeTaskName, - UUID: uuid, - UUIDSuffix: endSuffix, - Filename: fa.filename, - } - - if active.Earlier(safeRelayLog) { - safeRelayLog = active - } - - fa.safeRelayLog = safeRelayLog - - // discard newer UUIDs - uuids := make([]string, 0, len(fa.uuids)) - for _, uuid := range fa.uuids { - _, suffix, _ := utils.ParseSuffixForUUID(uuid) - if suffix > endSuffix { - break - } - uuids = append(uuids, uuid) - } - fa.uuids = uuids -} - -func (fa *filenameArgs) String() string { - return fmt.Sprintf("(RelayBaseDir: %s, Filename: %s, SubDir: %s, UUIDs: %s, SafeRelayLog: %s)", - fa.relayBaseDir, fa.filename, fa.subDir, strings.Join(fa.uuids, ";"), fa.safeRelayLog) -} - -// filenameStrategy represents a relay purge strategy by filename -// similar to `PURGE BINARY LOGS TO`. -type filenameStrategy struct { - purging atomic.Bool - - logger log.Logger -} - -func newFilenameStrategy() PurgeStrategy { - return &filenameStrategy{ - logger: log.With(zap.String("component", "relay purger"), zap.String("strategy", "file name")), - } -} - -func (s *filenameStrategy) Check(args interface{}) (bool, error) { - // do not support purge in the background - return false, nil -} - -func (s *filenameStrategy) Do(args interface{}) error { - if !s.purging.CAS(false, true) { - return terror.ErrRelayThisStrategyIsPurging.Generate() - } - defer s.purging.Store(false) - - fa, ok := args.(*filenameArgs) - if !ok { - return terror.ErrRelayPurgeArgsNotValid.Generate(args, args) - } - - return purgeRelayFilesBeforeFile(s.logger, fa.relayBaseDir, fa.uuids, fa.safeRelayLog) -} - -func (s *filenameStrategy) Purging() bool { - return s.purging.Load() -} - -func (s *filenameStrategy) Type() strategyType { - return strategyFilename -} diff --git a/dm/relay/purger/strategy_inactive.go b/dm/relay/purger/strategy_inactive.go deleted file mode 100644 index 0c404a2b29d..00000000000 --- a/dm/relay/purger/strategy_inactive.go +++ /dev/null @@ -1,86 +0,0 @@ -// Copyright 2019 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package purger - -import ( - "fmt" - "strings" - - "go.uber.org/atomic" - "go.uber.org/zap" - - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/streamer" - "github.com/pingcap/ticdc/dm/pkg/terror" -) - -// inactiveArgs represents args needed by inactiveStrategy. -type inactiveArgs struct { - relayBaseDir string - uuids []string - activeRelayLog *streamer.RelayLogInfo // earliest active relay log info -} - -func (ia *inactiveArgs) SetActiveRelayLog(active *streamer.RelayLogInfo) { - ia.activeRelayLog = active -} - -func (ia *inactiveArgs) String() string { - return fmt.Sprintf("(RelayBaseDir: %s, UUIDs: %s, ActiveRelayLog: %s)", - ia.relayBaseDir, strings.Join(ia.uuids, ";"), ia.activeRelayLog) -} - -// inactiveStrategy represents a relay purge strategy which purge all inactive relay log files -// definition of inactive relay log files: -// * not writing by relay unit -// * not reading by sync unit and will not be read by any running tasks -// TODO zxc: judge tasks are running dumper / loader -type inactiveStrategy struct { - purging atomic.Bool - - logger log.Logger -} - -func newInactiveStrategy() PurgeStrategy { - return &inactiveStrategy{ - logger: log.With(zap.String("component", "relay purger"), zap.String("strategy", "inactive binlog file")), - } -} - -func (s *inactiveStrategy) Check(args interface{}) (bool, error) { - // do not support purge in the background - return false, nil -} - -func (s *inactiveStrategy) Do(args interface{}) error { - if !s.purging.CAS(false, true) { - return terror.ErrRelayThisStrategyIsPurging.Generate() - } - defer s.purging.Store(false) - - ia, ok := args.(*inactiveArgs) - if !ok { - return terror.ErrRelayPurgeArgsNotValid.Generate(args, args) - } - - return purgeRelayFilesBeforeFile(s.logger, ia.relayBaseDir, ia.uuids, ia.activeRelayLog) -} - -func (s *inactiveStrategy) Purging() bool { - return s.purging.Load() -} - -func (s *inactiveStrategy) Type() strategyType { - return strategyInactive -} diff --git a/dm/relay/purger/strategy_space.go b/dm/relay/purger/strategy_space.go deleted file mode 100644 index ce281b01a35..00000000000 --- a/dm/relay/purger/strategy_space.go +++ /dev/null @@ -1,96 +0,0 @@ -// Copyright 2019 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package purger - -import ( - "fmt" - "strings" - - "go.uber.org/atomic" - "go.uber.org/zap" - - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/streamer" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" -) - -// spaceArgs represents args needed by spaceStrategy. -type spaceArgs struct { - relayBaseDir string - remainSpace int64 // if remain space (GB) in @RelayBaseDir less than this, then it can be purged - uuids []string - activeRelayLog *streamer.RelayLogInfo // earliest active relay log info -} - -func (sa *spaceArgs) SetActiveRelayLog(active *streamer.RelayLogInfo) { - sa.activeRelayLog = active -} - -func (sa *spaceArgs) String() string { - return fmt.Sprintf("(RelayBaseDir: %s, AllowMinRemainSpace: %dGB, UUIDs: %s, ActiveRelayLog: %s)", - sa.relayBaseDir, sa.remainSpace, strings.Join(sa.uuids, ";"), sa.activeRelayLog) -} - -// spaceStrategy represents a relay purge strategy by remain space in dm-worker node. -type spaceStrategy struct { - purging atomic.Bool - - logger log.Logger -} - -func newSpaceStrategy() PurgeStrategy { - return &spaceStrategy{ - logger: log.With(zap.String("component", "relay purger"), zap.String("strategy", "space")), - } -} - -func (s *spaceStrategy) Check(args interface{}) (bool, error) { - sa, ok := args.(*spaceArgs) - if !ok { - return false, terror.ErrRelayPurgeArgsNotValid.Generate(args, args) - } - - storageSize, err := utils.GetStorageSize(sa.relayBaseDir) - if err != nil { - return false, terror.Annotatef(err, "get storage size for directory %s", sa.relayBaseDir) - } - - requiredBytes := uint64(sa.remainSpace) * 1024 * 1024 * 1024 - return storageSize.Available < requiredBytes, nil -} - -func (s *spaceStrategy) Do(args interface{}) error { - if !s.purging.CAS(false, true) { - return terror.ErrRelayThisStrategyIsPurging.Generate() - } - defer s.purging.Store(false) - - sa, ok := args.(*spaceArgs) - if !ok { - return terror.ErrRelayPurgeArgsNotValid.Generate(args, args) - } - - // NOTE: we purge all inactive relay log files when available space less than @remainSpace - // maybe we can refine this to purge only part of this files every time - return purgeRelayFilesBeforeFile(s.logger, sa.relayBaseDir, sa.uuids, sa.activeRelayLog) -} - -func (s *spaceStrategy) Purging() bool { - return s.purging.Load() -} - -func (s *spaceStrategy) Type() strategyType { - return strategySpace -} diff --git a/dm/relay/purger/strategy_time.go b/dm/relay/purger/strategy_time.go deleted file mode 100644 index b52f143c37d..00000000000 --- a/dm/relay/purger/strategy_time.go +++ /dev/null @@ -1,88 +0,0 @@ -// Copyright 2019 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package purger - -import ( - "fmt" - "strings" - "time" - - "go.uber.org/atomic" - "go.uber.org/zap" - - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/streamer" - "github.com/pingcap/ticdc/dm/pkg/terror" -) - -// timeArgs represents args needed by timeStrategy. -type timeArgs struct { - relayBaseDir string - safeTime time.Time // if file's modified time is older than this, then it can be purged - uuids []string - activeRelayLog *streamer.RelayLogInfo // earliest active relay log info -} - -func (ta *timeArgs) SetActiveRelayLog(active *streamer.RelayLogInfo) { - ta.activeRelayLog = active -} - -func (ta *timeArgs) String() string { - return fmt.Sprintf("(RelayBaseDir: %s, SafeTime: %s, UUIDs: %s, ActiveRelayLog: %s)", - ta.relayBaseDir, ta.safeTime, strings.Join(ta.uuids, ";"), ta.activeRelayLog) -} - -// timeStrategy represents a relay purge strategy by time -// similar to `PURGE BINARY LOGS BEFORE` in MySQL. -type timeStrategy struct { - purging atomic.Bool - - logger log.Logger -} - -func newTimeStrategy() PurgeStrategy { - return &timeStrategy{ - logger: log.With(zap.String("component", "relay purger"), zap.String("strategy", "time")), - } -} - -func (s *timeStrategy) Check(args interface{}) (bool, error) { - // for time strategy, we always try to do the purging - return true, nil -} - -func (s *timeStrategy) Stop() { -} - -func (s *timeStrategy) Do(args interface{}) error { - if !s.purging.CAS(false, true) { - return terror.ErrRelayThisStrategyIsPurging.Generate() - } - defer s.purging.Store(false) - - ta, ok := args.(*timeArgs) - if !ok { - return terror.ErrRelayPurgeArgsNotValid.Generate(args, args) - } - - return purgeRelayFilesBeforeFileAndTime(s.logger, ta.relayBaseDir, ta.uuids, ta.activeRelayLog, ta.safeTime) -} - -func (s *timeStrategy) Purging() bool { - return s.purging.Load() -} - -func (s *timeStrategy) Type() strategyType { - return strategyTime -} diff --git a/dm/relay/purger/file.go b/dm/relay/purger_helper.go similarity index 97% rename from dm/relay/purger/file.go rename to dm/relay/purger_helper.go index 5e8c2cdb4d7..a840146ebc9 100644 --- a/dm/relay/purger/file.go +++ b/dm/relay/purger_helper.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package purger +package relay import ( "os" @@ -107,7 +107,7 @@ func collectRelayFilesBeforeFileAndTime(logger log.Logger, relayBaseDir string, ) if i+1 == len(uuids) { // same sub dir, only collect relay files newer than safeRelay.filename - shortFiles, err = streamer.CollectBinlogFilesCmp(dir, safeFilename, streamer.FileCmpLess) + shortFiles, err = CollectBinlogFilesCmp(dir, safeFilename, FileCmpLess) if err != nil { return nil, terror.Annotatef(err, "dir %s", dir) } @@ -117,7 +117,7 @@ func collectRelayFilesBeforeFileAndTime(logger log.Logger, relayBaseDir string, continue } // earlier sub dir, collect all relay files - shortFiles, err = streamer.CollectAllBinlogFiles(dir) + shortFiles, err = CollectAllBinlogFiles(dir) if err != nil { return nil, terror.Annotatef(err, "dir %s", dir) } diff --git a/dm/relay/purger/file_test.go b/dm/relay/purger_helper_test.go similarity index 99% rename from dm/relay/purger/file_test.go rename to dm/relay/purger_helper_test.go index c926bd47a35..cebc77abc62 100644 --- a/dm/relay/purger/file_test.go +++ b/dm/relay/purger_helper_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package purger +package relay import ( "os" diff --git a/dm/relay/purger/purger_test.go b/dm/relay/purger_test.go similarity index 95% rename from dm/relay/purger/purger_test.go rename to dm/relay/purger_test.go index c43def46daa..0919077c60f 100644 --- a/dm/relay/purger/purger_test.go +++ b/dm/relay/purger_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package purger +package relay import ( "bytes" @@ -42,14 +42,14 @@ var _ = Suite(&testPurgerSuite{ {"mysql-bin.000001", "mysql-bin.000002", "mysql-bin.000003"}, }, activeRelayLog: &streamer.RelayLogInfo{ - TaskName: fakeTaskName, + TaskName: fakeStrategyTaskName, UUID: "e9540a0d-f16d-11e8-8cb7-0242ac130008.000002", UUIDSuffix: 2, Filename: "mysql-bin.000003", // last in second sub dir }, }) -func TestSuite(t *testing.T) { +func TestPurgerSuite(t *testing.T) { TestingT(t) } @@ -82,7 +82,7 @@ func (t *testPurgerSuite) TestPurgeManuallyInactive(c *C) { Interval: 0, // disable automatically } - purger := NewPurger(cfg, baseDir, []RelayOperator{t}, nil) + purger := NewPurger(cfg, baseDir, []Operator{t}, nil) req := &pb.PurgeRelayRequest{ Inactive: true, @@ -121,7 +121,7 @@ func (t *testPurgerSuite) TestPurgeManuallyTime(c *C) { Interval: 0, // disable automatically } - purger := NewPurger(cfg, baseDir, []RelayOperator{t}, nil) + purger := NewPurger(cfg, baseDir, []Operator{t}, nil) req := &pb.PurgeRelayRequest{ Time: safeTime.Unix(), @@ -160,7 +160,7 @@ func (t *testPurgerSuite) TestPurgeManuallyFilename(c *C) { Interval: 0, // disable automatically } - purger := NewPurger(cfg, baseDir, []RelayOperator{t}, nil) + purger := NewPurger(cfg, baseDir, []Operator{t}, nil) req := &pb.PurgeRelayRequest{ Filename: t.relayFiles[0][2], @@ -214,7 +214,7 @@ func (t *testPurgerSuite) TestPurgeAutomaticallyTime(c *C) { } } - purger := NewPurger(cfg, baseDir, []RelayOperator{t}, nil) + purger := NewPurger(cfg, baseDir, []Operator{t}, nil) purger.Start() time.Sleep(2 * time.Second) // sleep enough time to purge all inactive relay log files purger.Close() @@ -254,7 +254,7 @@ func (t *testPurgerSuite) TestPurgeAutomaticallySpace(c *C) { RemainSpace: int64(storageSize.Available)/1024/1024/1024 + 1024, // always trigger purge } - purger := NewPurger(cfg, baseDir, []RelayOperator{t}, nil) + purger := NewPurger(cfg, baseDir, []Operator{t}, nil) purger.Start() time.Sleep(2 * time.Second) // sleep enough time to purge all inactive relay log files purger.Close() @@ -336,7 +336,7 @@ func (t *testPurgerSuite) TestPurgerInterceptor(c *C) { cfg := config.PurgeConfig{} interceptor := newFakeInterceptor() - purger := NewPurger(cfg, "", []RelayOperator{t}, []PurgeInterceptor{interceptor}) + purger := NewPurger(cfg, "", []Operator{t}, []PurgeInterceptor{interceptor}) req := &pb.PurgeRelayRequest{ Inactive: true, diff --git a/dm/relay/relay.go b/dm/relay/relay.go index 3db05c27de1..e0322a4ef41 100644 --- a/dm/relay/relay.go +++ b/dm/relay/relay.go @@ -44,14 +44,11 @@ import ( pkgstreamer "github.com/pingcap/ticdc/dm/pkg/streamer" "github.com/pingcap/ticdc/dm/pkg/terror" "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/relay/reader" - "github.com/pingcap/ticdc/dm/relay/retry" "github.com/pingcap/ticdc/dm/relay/transformer" - "github.com/pingcap/ticdc/dm/relay/writer" ) // used to fill RelayLogInfo. -var fakeTaskName = "relay" +var fakeRelayTaskName = "relay" const ( flushMetaInterval = 30 * time.Second @@ -295,7 +292,7 @@ func (r *Relay) process(ctx context.Context) error { } }() - readerRetry, err := retry.NewReaderRetry(r.cfg.ReaderRetry) + readerRetry, err := NewReaderRetry(r.cfg.ReaderRetry) if err != nil { return err } @@ -395,11 +392,11 @@ func (r *Relay) tryRecoverLatestFile(ctx context.Context, parser2 *parser.Parser } // setup a special writer to do the recovering - cfg := &writer.FileConfig{ + cfg := &FileConfig{ RelayDir: r.meta.Dir(), Filename: latestPos.Name, } - writer2 := writer.NewFileWriter(r.logger, cfg, parser2) + writer2 := NewFileWriter(r.logger, cfg, parser2) err := writer2.Start() if err != nil { return terror.Annotatef(err, "start recover writer for UUID %s with config %+v", uuid, cfg) @@ -462,9 +459,9 @@ func (r *Relay) tryRecoverLatestFile(ctx context.Context, parser2 *parser.Parser // the first return value is the index of last read rows event if the transaction is not finished. func (r *Relay) handleEvents( ctx context.Context, - reader2 reader.Reader, + reader2 Reader, transformer2 transformer.Transformer, - writer2 writer.Writer, + writer2 Writer, ) (int, error) { var ( _, lastPos = r.meta.Pos() @@ -811,7 +808,7 @@ func (r *Relay) doIntervalOps(ctx context.Context) { } // setUpReader setups the underlying reader used to read binlog events from the upstream master server. -func (r *Relay) setUpReader(ctx context.Context) (reader.Reader, error) { +func (r *Relay) setUpReader(ctx context.Context) (Reader, error) { ctx2, cancel := context.WithTimeout(ctx, utils.DefaultDBTimeout) defer cancel() @@ -826,7 +823,7 @@ func (r *Relay) setUpReader(ctx context.Context) (reader.Reader, error) { uuid, pos := r.meta.Pos() _, gs := r.meta.GTID() - cfg := &reader.Config{ + cfg := &RConfig{ SyncConfig: r.syncerCfg, Pos: pos, GTIDs: gs, @@ -834,7 +831,7 @@ func (r *Relay) setUpReader(ctx context.Context) (reader.Reader, error) { EnableGTID: r.cfg.EnableGTID, } - reader2 := reader.NewReader(cfg) + reader2 := NewUpstreamReader(cfg) err = reader2.Start() if err != nil { // do not log the whole config to protect the password in `SyncConfig`. @@ -847,13 +844,13 @@ func (r *Relay) setUpReader(ctx context.Context) (reader.Reader, error) { } // setUpWriter setups the underlying writer used to writer binlog events into file or other places. -func (r *Relay) setUpWriter(parser2 *parser.Parser) (writer.Writer, error) { +func (r *Relay) setUpWriter(parser2 *parser.Parser) (Writer, error) { uuid, pos := r.meta.Pos() - cfg := &writer.FileConfig{ + cfg := &FileConfig{ RelayDir: r.meta.Dir(), Filename: pos.Name, } - writer2 := writer.NewFileWriter(r.logger, cfg, parser2) + writer2 := NewFileWriter(r.logger, cfg, parser2) if err := writer2.Start(); err != nil { return nil, terror.Annotatef(err, "start writer for UUID %s with config %+v", uuid, cfg) } @@ -1027,7 +1024,7 @@ func (r *Relay) setActiveRelayLog(filename string) { uuid := r.meta.UUID() _, suffix, _ := utils.ParseSuffixForUUID(uuid) rli := &pkgstreamer.RelayLogInfo{ - TaskName: fakeTaskName, + TaskName: fakeRelayTaskName, UUID: uuid, UUIDSuffix: suffix, Filename: filename, diff --git a/dm/relay/relay_test.go b/dm/relay/relay_test.go index 3cb01ec80be..30a5176777c 100644 --- a/dm/relay/relay_test.go +++ b/dm/relay/relay_test.go @@ -36,12 +36,8 @@ import ( "github.com/pingcap/ticdc/dm/pkg/binlog/event" "github.com/pingcap/ticdc/dm/pkg/conn" "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/log" "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/relay/reader" - "github.com/pingcap/ticdc/dm/relay/retry" "github.com/pingcap/ticdc/dm/relay/transformer" - "github.com/pingcap/ticdc/dm/relay/writer" ) var _ = Suite(&testRelaySuite{}) @@ -52,10 +48,6 @@ func TestSuite(t *testing.T) { type testRelaySuite struct{} -func (t *testRelaySuite) SetUpSuite(c *C) { - c.Assert(log.InitLogger(&log.Config{}), IsNil) -} - func newRelayCfg(c *C, flavor string) *Config { dbCfg := getDBConfigForTest() return &Config{ @@ -69,7 +61,7 @@ func newRelayCfg(c *C, flavor string) *Config { User: dbCfg.User, Password: dbCfg.Password, }, - ReaderRetry: retry.ReaderRetryConfig{ + ReaderRetry: ReaderRetryConfig{ BackoffRollback: 200 * time.Millisecond, BackoffMax: 1 * time.Second, BackoffMin: 1 * time.Millisecond, @@ -103,7 +95,7 @@ func getDBConfigForTest() config.DBConfig { // mockReader is used only for relay testing. type mockReader struct { - result reader.Result + result RResult err error } @@ -115,10 +107,10 @@ func (r *mockReader) Close() error { return nil } -func (r *mockReader) GetEvent(ctx context.Context) (reader.Result, error) { +func (r *mockReader) GetEvent(ctx context.Context) (RResult, error) { select { case <-ctx.Done(): - return reader.Result{}, ctx.Err() + return RResult{}, ctx.Err() default: } return r.result, r.err @@ -126,7 +118,7 @@ func (r *mockReader) GetEvent(ctx context.Context) (reader.Result, error) { // mockWriter is used only for relay testing. type mockWriter struct { - result writer.Result + result WResult err error latestEvent *replication.BinlogEvent } @@ -139,11 +131,11 @@ func (w *mockWriter) Close() error { return nil } -func (w *mockWriter) Recover(ctx context.Context) (writer.RecoverResult, error) { - return writer.RecoverResult{}, nil +func (w *mockWriter) Recover(ctx context.Context) (RecoverResult, error) { + return RecoverResult{}, nil } -func (w *mockWriter) WriteEvent(ev *replication.BinlogEvent) (writer.Result, error) { +func (w *mockWriter) WriteEvent(ev *replication.BinlogEvent) (WResult, error) { w.latestEvent = ev // hold it return w.result, w.err } diff --git a/dm/relay/writer/file.go b/dm/relay/relay_writer.go similarity index 77% rename from dm/relay/writer/file.go rename to dm/relay/relay_writer.go index fc444245299..0cb0b52ac06 100644 --- a/dm/relay/writer/file.go +++ b/dm/relay/relay_writer.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package writer +package relay import ( "context" @@ -30,12 +30,62 @@ import ( "github.com/pingcap/ticdc/dm/pkg/binlog" "github.com/pingcap/ticdc/dm/pkg/binlog/common" "github.com/pingcap/ticdc/dm/pkg/binlog/event" - bw "github.com/pingcap/ticdc/dm/pkg/binlog/writer" + "github.com/pingcap/ticdc/dm/pkg/gtid" "github.com/pingcap/ticdc/dm/pkg/log" "github.com/pingcap/ticdc/dm/pkg/terror" "github.com/pingcap/ticdc/dm/pkg/utils" ) +const ( + ignoreReasonAlreadyExists = "already exists" + ignoreReasonFakeRotate = "fake rotate event" +) + +// WResult represents a write result. +type WResult struct { + Ignore bool // whether the event ignored by the writer + IgnoreReason string // why the writer ignore the event +} + +// RecoverResult represents a result for a binlog recover operation. +type RecoverResult struct { + // if truncate trailing incomplete events during recovering in relay log + Truncated bool + // the latest binlog position after recover operation has done. + LatestPos gmysql.Position + // the latest binlog GTID set after recover operation has done. + LatestGTIDs gtid.Set +} + +// Writer writes binlog events into disk or any other memory structure. +// The writer should support: +// 1. write binlog events and report the operation result +// 2. skip any obsolete binlog events +// 3. generate dummy events to fill the gap if needed +// 4. rotate binlog(relay) file if needed +// 5. rollback/discard unfinished binlog entries(events or transactions) +type Writer interface { + // Start prepares the writer for writing binlog events. + Start() error + + // Close closes the writer and release the resource. + Close() error + + // Recover tries to recover the binlog file or any other memory structure associate with this writer. + // It is often used to recover a binlog file with some corrupt/incomplete binlog events/transactions at the end of the file. + // It is not safe for concurrent use by multiple goroutines. + // It should be called before writing to the file. + Recover(ctx context.Context) (RecoverResult, error) + + // WriteEvent writes an binlog event's data into disk or any other places. + // It is not safe for concurrent use by multiple goroutines. + WriteEvent(ev *replication.BinlogEvent) (WResult, error) + + // Flush flushes the buffered data to a stable storage or sends through the network. + // It is not safe for concurrent use by multiple goroutines. + Flush() error +} + // FileConfig is the configuration used by the FileWriter. type FileConfig struct { RelayDir string // directory to store relay log files. @@ -51,7 +101,7 @@ type FileWriter struct { // underlying binlog writer, // it will be created/started until needed. - out *bw.FileWriter + out *BinlogWriter // the parser often used to verify events's statement through parsing them. parser *parser.Parser @@ -116,12 +166,12 @@ func (w *FileWriter) Recover(ctx context.Context) (RecoverResult, error) { } // WriteEvent implements Writer.WriteEvent. -func (w *FileWriter) WriteEvent(ev *replication.BinlogEvent) (Result, error) { +func (w *FileWriter) WriteEvent(ev *replication.BinlogEvent) (WResult, error) { w.mu.Lock() defer w.mu.Unlock() if w.stage != common.StagePrepared { - return Result{}, terror.ErrRelayWriterNeedStart.Generate(w.stage, common.StagePrepared) + return WResult{}, terror.ErrRelayWriterNeedStart.Generate(w.stage, common.StagePrepared) } switch ev.Event.(type) { @@ -155,7 +205,7 @@ func (w *FileWriter) offset() int64 { if w.out == nil { return 0 } - status := w.out.Status().(*bw.FileWriterStatus) + status := w.out.Status().(*BinlogWriterStatus) return status.Offset } @@ -164,53 +214,53 @@ func (w *FileWriter) offset() int64 { // 2. open/create a new binlog file // 3. write the binlog file header if not exists // 4. write the FormatDescriptionEvent if not exists one -func (w *FileWriter) handleFormatDescriptionEvent(ev *replication.BinlogEvent) (Result, error) { +func (w *FileWriter) handleFormatDescriptionEvent(ev *replication.BinlogEvent) (WResult, error) { // close the previous binlog file if w.out != nil { w.logger.Info("closing previous underlying binlog writer", zap.Reflect("status", w.out.Status())) err := w.out.Close() if err != nil { - return Result{}, terror.Annotate(err, "close previous underlying binlog writer") + return WResult{}, terror.Annotate(err, "close previous underlying binlog writer") } } // verify filename if !binlog.VerifyFilename(w.filename.Load()) { - return Result{}, terror.ErrRelayBinlogNameNotValid.Generatef("binlog filename %s not valid", w.filename.Load()) + return WResult{}, terror.ErrRelayBinlogNameNotValid.Generatef("binlog filename %s not valid", w.filename.Load()) } // open/create a new binlog file filename := filepath.Join(w.cfg.RelayDir, w.filename.Load()) - outCfg := &bw.FileWriterConfig{ + outCfg := &BinlogWriterConfig{ Filename: filename, } - out := bw.NewFileWriter(w.logger, outCfg) + out := NewBinlogWriter(w.logger, outCfg) err := out.Start() if err != nil { - return Result{}, terror.Annotatef(err, "start underlying binlog writer for %s", filename) + return WResult{}, terror.Annotatef(err, "start underlying binlog writer for %s", filename) } - w.out = out.(*bw.FileWriter) + w.out = out w.logger.Info("open underlying binlog writer", zap.Reflect("status", w.out.Status())) // write the binlog file header if not exists exist, err := checkBinlogHeaderExist(filename) if err != nil { - return Result{}, terror.Annotatef(err, "check binlog file header for %s", filename) + return WResult{}, terror.Annotatef(err, "check binlog file header for %s", filename) } else if !exist { err = w.out.Write(replication.BinLogFileHeader) if err != nil { - return Result{}, terror.Annotatef(err, "write binlog file header for %s", filename) + return WResult{}, terror.Annotatef(err, "write binlog file header for %s", filename) } } // write the FormatDescriptionEvent if not exists one exist, err = checkFormatDescriptionEventExist(filename) if err != nil { - return Result{}, terror.Annotatef(err, "check FormatDescriptionEvent for %s", filename) + return WResult{}, terror.Annotatef(err, "check FormatDescriptionEvent for %s", filename) } else if !exist { err = w.out.Write(ev.RawData) if err != nil { - return Result{}, terror.Annotatef(err, "write FormatDescriptionEvent %+v for %s", ev.Header, filename) + return WResult{}, terror.Annotatef(err, "write FormatDescriptionEvent %+v for %s", ev.Header, filename) } } var reason string @@ -218,7 +268,7 @@ func (w *FileWriter) handleFormatDescriptionEvent(ev *replication.BinlogEvent) ( reason = ignoreReasonAlreadyExists } - return Result{ + return WResult{ Ignore: exist, // ignore if exists IgnoreReason: reason, }, nil @@ -232,7 +282,7 @@ func (w *FileWriter) handleFormatDescriptionEvent(ev *replication.BinlogEvent) ( // NOTE: we do not create a new binlog file when received a RotateEvent, // instead, we create a new binlog file when received a FormatDescriptionEvent. // because a binlog file without any events has no meaning. -func (w *FileWriter) handleRotateEvent(ev *replication.BinlogEvent) (result Result, err error) { +func (w *FileWriter) handleRotateEvent(ev *replication.BinlogEvent) (result WResult, err error) { rotateEv, ok := ev.Event.(*replication.RotateEvent) if !ok { return result, terror.ErrRelayWriterExpectRotateEv.Generate(ev.Header) @@ -256,7 +306,7 @@ func (w *FileWriter) handleRotateEvent(ev *replication.BinlogEvent) (result Resu // write the RotateEvent if not fake if utils.IsFakeRotateEvent(ev.Header) { // skip fake rotate event - return Result{ + return WResult{ Ignore: true, IgnoreReason: ignoreReasonFakeRotate, }, nil @@ -277,7 +327,7 @@ func (w *FileWriter) handleRotateEvent(ev *replication.BinlogEvent) (result Resu return result, terror.Annotatef(err, "write RotateEvent %+v for %s", ev.Header, filepath.Join(w.cfg.RelayDir, currFile)) } - return Result{ + return WResult{ Ignore: false, }, nil } @@ -286,27 +336,27 @@ func (w *FileWriter) handleRotateEvent(ev *replication.BinlogEvent) (result Resu // 1. handle a potential hole if exists // 2. handle any duplicate events if exist // 3. write the non-duplicate event -func (w *FileWriter) handleEventDefault(ev *replication.BinlogEvent) (Result, error) { +func (w *FileWriter) handleEventDefault(ev *replication.BinlogEvent) (WResult, error) { result, err := w.handlePotentialHoleOrDuplicate(ev) if err != nil { - return Result{}, err + return WResult{}, err } else if result.Ignore { return result, nil } // write the non-duplicate event err = w.out.Write(ev.RawData) - return Result{ + return WResult{ Ignore: false, }, terror.Annotatef(err, "write event %+v", ev.Header) } // handlePotentialHoleOrDuplicate combines handleFileHoleExist and handleDuplicateEventsExist. -func (w *FileWriter) handlePotentialHoleOrDuplicate(ev *replication.BinlogEvent) (Result, error) { +func (w *FileWriter) handlePotentialHoleOrDuplicate(ev *replication.BinlogEvent) (WResult, error) { // handle a potential hole mayDuplicate, err := w.handleFileHoleExist(ev) if err != nil { - return Result{}, terror.Annotatef(err, "handle a potential hole in %s before %+v", + return WResult{}, terror.Annotatef(err, "handle a potential hole in %s before %+v", w.filename.Load(), ev.Header) } @@ -314,7 +364,7 @@ func (w *FileWriter) handlePotentialHoleOrDuplicate(ev *replication.BinlogEvent) // handle any duplicate events if exist result, err2 := w.handleDuplicateEventsExist(ev) if err2 != nil { - return Result{}, terror.Annotatef(err2, "handle a potential duplicate event %+v in %s", + return WResult{}, terror.Annotatef(err2, "handle a potential duplicate event %+v in %s", ev.Header, w.filename.Load()) } if result.Ignore { @@ -323,7 +373,7 @@ func (w *FileWriter) handlePotentialHoleOrDuplicate(ev *replication.BinlogEvent) } } - return Result{ + return WResult{ Ignore: false, }, nil } @@ -335,7 +385,7 @@ func (w *FileWriter) handlePotentialHoleOrDuplicate(ev *replication.BinlogEvent) func (w *FileWriter) handleFileHoleExist(ev *replication.BinlogEvent) (bool, error) { // 1. detect whether a hole exists evStartPos := int64(ev.Header.LogPos - ev.Header.EventSize) - outFs, ok := w.out.Status().(*bw.FileWriterStatus) + outFs, ok := w.out.Status().(*BinlogWriterStatus) if !ok { return false, terror.ErrRelayWriterStatusNotValid.Generate(w.out.Status()) } @@ -367,11 +417,11 @@ func (w *FileWriter) handleFileHoleExist(ev *replication.BinlogEvent) (bool, err } // handleDuplicateEventsExist tries to handle a potential duplicate event in the binlog file. -func (w *FileWriter) handleDuplicateEventsExist(ev *replication.BinlogEvent) (Result, error) { +func (w *FileWriter) handleDuplicateEventsExist(ev *replication.BinlogEvent) (WResult, error) { filename := filepath.Join(w.cfg.RelayDir, w.filename.Load()) duplicate, err := checkIsDuplicateEvent(filename, ev) if err != nil { - return Result{}, terror.Annotatef(err, "check event %+v whether duplicate in %s", ev.Header, filename) + return WResult{}, terror.Annotatef(err, "check event %+v whether duplicate in %s", ev.Header, filename) } else if duplicate { w.logger.Info("event is duplicate", zap.Reflect("header", ev.Header), zap.String("file", w.filename.Load())) } @@ -381,7 +431,7 @@ func (w *FileWriter) handleDuplicateEventsExist(ev *replication.BinlogEvent) (Re reason = ignoreReasonAlreadyExists } - return Result{ + return WResult{ Ignore: duplicate, IgnoreReason: reason, }, nil diff --git a/dm/relay/writer/file_test.go b/dm/relay/relay_writer_test.go similarity index 99% rename from dm/relay/writer/file_test.go rename to dm/relay/relay_writer_test.go index 685672facb8..3779afd420f 100644 --- a/dm/relay/writer/file_test.go +++ b/dm/relay/relay_writer_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package writer +package relay import ( "bytes" @@ -35,7 +35,7 @@ import ( var _ = check.Suite(&testFileWriterSuite{}) -func TestSuite(t *testing.T) { +func TestFileWriterSuite(t *testing.T) { check.TestingT(t) } diff --git a/dm/relay/retry/reader.go b/dm/relay/remote_retry.go similarity index 99% rename from dm/relay/retry/reader.go rename to dm/relay/remote_retry.go index c9e37e00c95..8b93c6b91c8 100644 --- a/dm/relay/retry/reader.go +++ b/dm/relay/remote_retry.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package retry +package relay import ( "context" diff --git a/dm/relay/retry/reader_test.go b/dm/relay/remote_retry_test.go similarity index 97% rename from dm/relay/retry/reader_test.go rename to dm/relay/remote_retry_test.go index 4f03db06314..20754274d09 100644 --- a/dm/relay/retry/reader_test.go +++ b/dm/relay/remote_retry_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package retry +package relay import ( "context" @@ -23,7 +23,7 @@ import ( . "github.com/pingcap/check" ) -func TestSuite(t *testing.T) { +func TestReaderRetrySuite(t *testing.T) { TestingT(t) } diff --git a/dm/pkg/streamer/streamer.go b/dm/relay/streamer.go similarity index 94% rename from dm/pkg/streamer/streamer.go rename to dm/relay/streamer.go index a7e4b67db06..891dbda1d3f 100644 --- a/dm/pkg/streamer/streamer.go +++ b/dm/relay/streamer.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package streamer +package relay import ( "bytes" @@ -20,7 +20,6 @@ import ( "github.com/pingcap/ticdc/dm/pkg/binlog/common" "github.com/pingcap/ticdc/dm/pkg/binlog/event" - "github.com/pingcap/ticdc/dm/pkg/binlog/reader" "github.com/pingcap/ticdc/dm/pkg/log" "github.com/pingcap/ticdc/dm/pkg/terror" @@ -33,9 +32,6 @@ var heartbeatInterval = common.MasterHeartbeatPeriod // TODO: maybe one day we can make a pull request to go-mysql to support LocalStreamer. -// Streamer provides the ability to get binlog event from remote server or local file. -type Streamer reader.Streamer - // LocalStreamer reads and parses binlog events from local binlog file. type LocalStreamer struct { ch chan *replication.BinlogEvent diff --git a/dm/pkg/streamer/streamer_test.go b/dm/relay/streamer_test.go similarity index 86% rename from dm/pkg/streamer/streamer_test.go rename to dm/relay/streamer_test.go index 279d874fb92..a3c59e33cd8 100644 --- a/dm/pkg/streamer/streamer_test.go +++ b/dm/relay/streamer_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package streamer +package relay import ( "context" @@ -31,9 +31,9 @@ var _ = Suite(&testStreamerSuite{}) type testStreamerSuite struct{} func (t *testStreamerSuite) TestStreamer(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/pkg/streamer/SetHeartbeatInterval", "return(10000)"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/relay/SetHeartbeatInterval", "return(10000)"), IsNil) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/pkg/streamer/SetHeartbeatInterval"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/relay/SetHeartbeatInterval"), IsNil) }() ctx, cancel := context.WithTimeout(context.Background(), time.Second) @@ -105,9 +105,9 @@ func (t *testStreamerSuite) TestStreamer(c *C) { } func (t *testStreamerSuite) TestHeartbeat(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/pkg/streamer/SetHeartbeatInterval", "return(1)"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/relay/SetHeartbeatInterval", "return(1)"), IsNil) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/pkg/streamer/SetHeartbeatInterval"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/relay/SetHeartbeatInterval"), IsNil) }() ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second) diff --git a/dm/relay/reader/reader.go b/dm/relay/upstream_reader.go similarity index 85% rename from dm/relay/reader/reader.go rename to dm/relay/upstream_reader.go index e928ccdd823..240cd26ca11 100644 --- a/dm/relay/reader/reader.go +++ b/dm/relay/upstream_reader.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package reader +package relay import ( "context" @@ -28,8 +28,8 @@ import ( "github.com/pingcap/ticdc/dm/pkg/terror" ) -// Result represents a read operation result. -type Result struct { +// RResult represents a read operation result. +type RResult struct { Event *replication.BinlogEvent } @@ -48,11 +48,11 @@ type Reader interface { // GetEvent gets the binlog event one by one, it will block if no event can be read. // You can pass a context (like Cancel) to break the block. - GetEvent(ctx context.Context) (Result, error) + GetEvent(ctx context.Context) (RResult, error) } -// Config is the configuration used by the Reader. -type Config struct { +// RConfig is the configuration used by the Reader. +type RConfig struct { SyncConfig replication.BinlogSyncerConfig Pos mysql.Position GTIDs gtid.Set @@ -61,8 +61,8 @@ type Config struct { } // reader implements Reader interface. -type reader struct { - cfg *Config +type upstreamReader struct { + cfg *RConfig mu sync.RWMutex stage common.Stage @@ -73,9 +73,9 @@ type reader struct { logger log.Logger } -// NewReader creates a Reader instance. -func NewReader(cfg *Config) Reader { - return &reader{ +// NewUpstreamReader creates a Reader instance. +func NewUpstreamReader(cfg *RConfig) Reader { + return &upstreamReader{ cfg: cfg, in: br.NewTCPReader(cfg.SyncConfig), out: make(chan *replication.BinlogEvent), @@ -84,7 +84,7 @@ func NewReader(cfg *Config) Reader { } // Start implements Reader.Start. -func (r *reader) Start() error { +func (r *upstreamReader) Start() error { r.mu.Lock() defer r.mu.Unlock() @@ -109,7 +109,7 @@ func (r *reader) Start() error { } // Close implements Reader.Close. -func (r *reader) Close() error { +func (r *upstreamReader) Close() error { r.mu.Lock() defer r.mu.Unlock() @@ -124,11 +124,11 @@ func (r *reader) Close() error { // GetEvent implements Reader.GetEvent. // NOTE: can only close the reader after this returned. -func (r *reader) GetEvent(ctx context.Context) (Result, error) { +func (r *upstreamReader) GetEvent(ctx context.Context) (RResult, error) { r.mu.RLock() defer r.mu.RUnlock() - var result Result + var result RResult if r.stage != common.StagePrepared { return result, terror.ErrRelayReaderNeedStart.Generate(r.stage, common.StagePrepared) } @@ -148,13 +148,13 @@ func (r *reader) GetEvent(ctx context.Context) (Result, error) { } } -func (r *reader) setUpReaderByGTID() error { +func (r *upstreamReader) setUpReaderByGTID() error { gs := r.cfg.GTIDs r.logger.Info("start sync", zap.String("master", r.cfg.MasterID), zap.Stringer("from GTID set", gs)) return r.in.StartSyncByGTID(gs) } -func (r *reader) setUpReaderByPos() error { +func (r *upstreamReader) setUpReaderByPos() error { pos := r.cfg.Pos r.logger.Info("start sync", zap.String("master", r.cfg.MasterID), zap.Stringer("from position", pos)) return r.in.StartSyncByPos(pos) diff --git a/dm/relay/reader/reader_test.go b/dm/relay/upstream_reader_test.go similarity index 84% rename from dm/relay/reader/reader_test.go rename to dm/relay/upstream_reader_test.go index e45f1d4eb76..81cd77232c7 100644 --- a/dm/relay/reader/reader_test.go +++ b/dm/relay/upstream_reader_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package reader +package relay import ( "context" @@ -25,22 +25,22 @@ import ( br "github.com/pingcap/ticdc/dm/pkg/binlog/reader" ) -var _ = check.Suite(&testReaderSuite{}) +var _ = check.Suite(&testRemoteReaderSuite{}) -func TestSuite(t *testing.T) { +func TestRemoteReaderSuite(t *testing.T) { check.TestingT(t) } -type testReaderSuite struct{} +type testRemoteReaderSuite struct{} -func (t *testReaderSuite) TestInterface(c *check.C) { +func (t *testRemoteReaderSuite) TestInterface(c *check.C) { cases := []*replication.BinlogEvent{ {RawData: []byte{1}}, {RawData: []byte{2}}, {RawData: []byte{3}}, } - cfg := &Config{ + cfg := &RConfig{ SyncConfig: replication.BinlogSyncerConfig{ ServerID: 101, }, @@ -48,18 +48,18 @@ func (t *testReaderSuite) TestInterface(c *check.C) { } // test with position - r := NewReader(cfg) + r := NewUpstreamReader(cfg) t.testInterfaceWithReader(c, r, cases) // test with GTID cfg.EnableGTID = true - r = NewReader(cfg) + r = NewUpstreamReader(cfg) t.testInterfaceWithReader(c, r, cases) } -func (t *testReaderSuite) testInterfaceWithReader(c *check.C, r Reader, cases []*replication.BinlogEvent) { +func (t *testRemoteReaderSuite) testInterfaceWithReader(c *check.C, r Reader, cases []*replication.BinlogEvent) { // replace underlying reader with a mock reader for testing - concreteR := r.(*reader) + concreteR := r.(*upstreamReader) c.Assert(concreteR, check.NotNil) mockR := br.NewMockReader() concreteR.in = mockR @@ -102,17 +102,17 @@ func (t *testReaderSuite) testInterfaceWithReader(c *check.C, r Reader, cases [] c.Assert(result.Event, check.IsNil) } -func (t *testReaderSuite) TestGetEventWithError(c *check.C) { - cfg := &Config{ +func (t *testRemoteReaderSuite) TestGetEventWithError(c *check.C) { + cfg := &RConfig{ SyncConfig: replication.BinlogSyncerConfig{ ServerID: 101, }, MasterID: "test-master", } - r := NewReader(cfg) + r := NewUpstreamReader(cfg) // replace underlying reader with a mock reader for testing - concreteR := r.(*reader) + concreteR := r.(*upstreamReader) c.Assert(concreteR, check.NotNil) mockR := br.NewMockReader() concreteR.in = mockR diff --git a/dm/relay/util.go b/dm/relay/util.go index 0f1b5e67906..dddb75d13d4 100644 --- a/dm/relay/util.go +++ b/dm/relay/util.go @@ -16,8 +16,12 @@ package relay import ( "context" "database/sql" + "io" "strings" + "github.com/pingcap/errors" + + "github.com/pingcap/ticdc/dm/pkg/terror" "github.com/pingcap/ticdc/dm/pkg/utils" ) @@ -37,3 +41,34 @@ func isNewServer(ctx context.Context, prevUUID string, db *sql.DB, flavor string } return true, nil } + +// getNextUUID gets (the nextUUID and its suffix) after the current UUID. +func getNextUUID(currUUID string, uuids []string) (string, string, error) { + for i := len(uuids) - 2; i >= 0; i-- { + if uuids[i] == currUUID { + nextUUID := uuids[i+1] + _, suffixInt, err := utils.ParseSuffixForUUID(nextUUID) + if err != nil { + return "", "", terror.Annotatef(err, "UUID %s", nextUUID) + } + return nextUUID, utils.SuffixIntToStr(suffixInt), nil + } + } + return "", "", nil +} + +// isIgnorableParseError checks whether is a ignorable error for `BinlogParser.ParseFile`. +func isIgnorableParseError(err error) bool { + if err == nil { + return false + } + + if strings.Contains(err.Error(), "err EOF") { + // NOTE: go-mysql returned err not includes caused err, but as message, ref: parser.go `parseSingleEvent` + return true + } else if errors.Cause(err) == io.EOF { + return true + } + + return false +} diff --git a/dm/relay/util_test.go b/dm/relay/util_test.go index cef391a13b1..834051054fd 100644 --- a/dm/relay/util_test.go +++ b/dm/relay/util_test.go @@ -16,6 +16,9 @@ package relay import ( "context" "fmt" + "io" + + "github.com/pingcap/errors" "github.com/DATA-DOG/go-sqlmock" gmysql "github.com/go-mysql-org/go-mysql/mysql" @@ -74,3 +77,97 @@ func mockGetRandomServerID(mockDB sqlmock.Sqlmock) { mockDB.ExpectQuery("SHOW GLOBAL VARIABLES LIKE 'server_id'").WillReturnRows( sqlmock.NewRows([]string{"Variable_name", "Value"}).AddRow("server_id", "1")) } + +func (t *testUtilSuite) TestGetNextUUID(c *C) { + UUIDs := []string{ + "b60868af-5a6f-11e9-9ea3-0242ac160006.000001", + "7acfedb5-3008-4fa2-9776-6bac42b025fe.000002", + "92ffd03b-813e-4391-b16a-177524e8d531.000003", + "338513ce-b24e-4ff8-9ded-9ac5aa8f4d74.000004", + } + cases := []struct { + currUUID string + UUIDs []string + nextUUID string + nextUUIDSuffix string + errMsgReg string + }{ + { + // empty current and UUID list + }, + { + // non-empty current UUID, but empty UUID list + currUUID: "b60868af-5a6f-11e9-9ea3-0242ac160006.000001", + }, + { + // empty current UUID, but non-empty UUID list + UUIDs: UUIDs, + }, + { + // current UUID in UUID list, has next UUID + currUUID: UUIDs[0], + UUIDs: UUIDs, + nextUUID: UUIDs[1], + nextUUIDSuffix: UUIDs[1][len(UUIDs[1])-6:], + }, + { + // current UUID in UUID list, but has no next UUID + currUUID: UUIDs[len(UUIDs)-1], + UUIDs: UUIDs, + }, + { + // current UUID not in UUID list + currUUID: "40ed16c1-f6f7-4012-aa9b-d360261d2b22.666666", + UUIDs: UUIDs, + }, + { + // invalid next UUID in UUID list + currUUID: UUIDs[len(UUIDs)-1], + UUIDs: append(UUIDs, "invalid-uuid"), + errMsgReg: ".*invalid-uuid.*", + }, + } + + for _, cs := range cases { + nu, nus, err := getNextUUID(cs.currUUID, cs.UUIDs) + if len(cs.errMsgReg) > 0 { + c.Assert(err, ErrorMatches, cs.errMsgReg) + } else { + c.Assert(err, IsNil) + } + c.Assert(nu, Equals, cs.nextUUID) + c.Assert(nus, Equals, cs.nextUUIDSuffix) + } +} + +func (t *testUtilSuite) TestIsIgnorableParseError(c *C) { + cases := []struct { + err error + ignorable bool + }{ + { + err: nil, + ignorable: false, + }, + { + err: io.EOF, + ignorable: true, + }, + { + err: errors.Annotate(io.EOF, "annotated end of file"), + ignorable: true, + }, + { + err: errors.New("get event header err EOF xxxx"), + ignorable: true, + }, + { + err: errors.New("some other error"), + ignorable: false, + }, + } + + for _, cs := range cases { + c.Assert(isIgnorableParseError(cs.err), Equals, cs.ignorable) + } +} diff --git a/dm/relay/writer/writer.go b/dm/relay/writer/writer.go deleted file mode 100644 index c06939f92ae..00000000000 --- a/dm/relay/writer/writer.go +++ /dev/null @@ -1,73 +0,0 @@ -// Copyright 2019 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package writer - -import ( - "context" - - gmysql "github.com/go-mysql-org/go-mysql/mysql" - "github.com/go-mysql-org/go-mysql/replication" - - "github.com/pingcap/ticdc/dm/pkg/gtid" -) - -const ( - ignoreReasonAlreadyExists = "already exists" - ignoreReasonFakeRotate = "fake rotate event" -) - -// Result represents a write result. -type Result struct { - Ignore bool // whether the event ignored by the writer - IgnoreReason string // why the writer ignore the event -} - -// RecoverResult represents a result for a binlog recover operation. -type RecoverResult struct { - // if truncate trailing incomplete events during recovering in relay log - Truncated bool - // the latest binlog position after recover operation has done. - LatestPos gmysql.Position - // the latest binlog GTID set after recover operation has done. - LatestGTIDs gtid.Set -} - -// Writer writes binlog events into disk or any other memory structure. -// The writer should support: -// 1. write binlog events and report the operation result -// 2. skip any obsolete binlog events -// 3. generate dummy events to fill the gap if needed -// 4. rotate binlog(relay) file if needed -// 5. rollback/discard unfinished binlog entries(events or transactions) -type Writer interface { - // Start prepares the writer for writing binlog events. - Start() error - - // Close closes the writer and release the resource. - Close() error - - // Recover tries to recover the binlog file or any other memory structure associate with this writer. - // It is often used to recover a binlog file with some corrupt/incomplete binlog events/transactions at the end of the file. - // It is not safe for concurrent use by multiple goroutines. - // It should be called before writing to the file. - Recover(ctx context.Context) (RecoverResult, error) - - // WriteEvent writes an binlog event's data into disk or any other places. - // It is not safe for concurrent use by multiple goroutines. - WriteEvent(ev *replication.BinlogEvent) (Result, error) - - // Flush flushes the buffered data to a stable storage or sends through the network. - // It is not safe for concurrent use by multiple goroutines. - Flush() error -} diff --git a/dm/syncer/streamer_controller.go b/dm/syncer/streamer_controller.go index 063b8950903..a4c3db51360 100644 --- a/dm/syncer/streamer_controller.go +++ b/dm/syncer/streamer_controller.go @@ -27,12 +27,13 @@ import ( "github.com/pingcap/ticdc/dm/pkg/binlog" "github.com/pingcap/ticdc/dm/pkg/binlog/common" + "github.com/pingcap/ticdc/dm/pkg/binlog/reader" tcontext "github.com/pingcap/ticdc/dm/pkg/context" "github.com/pingcap/ticdc/dm/pkg/log" "github.com/pingcap/ticdc/dm/pkg/retry" - "github.com/pingcap/ticdc/dm/pkg/streamer" "github.com/pingcap/ticdc/dm/pkg/terror" "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/ticdc/dm/relay" "github.com/pingcap/ticdc/dm/syncer/dbconn" ) @@ -46,16 +47,16 @@ var minErrorRetryInterval = 1 * time.Minute // For other implementations who implement StreamerProducer and Streamer can easily take place of Syncer.streamProducer // For test is easy to mock. type StreamerProducer interface { - generateStreamer(location binlog.Location) (streamer.Streamer, error) + generateStreamer(location binlog.Location) (reader.Streamer, error) } // Read local relay log. type localBinlogReader struct { - reader *streamer.BinlogReader + reader *relay.BinlogReader EnableGTID bool } -func (l *localBinlogReader) generateStreamer(location binlog.Location) (streamer.Streamer, error) { +func (l *localBinlogReader) generateStreamer(location binlog.Location) (reader.Streamer, error) { if l.EnableGTID { return l.reader.StartSyncByGTID(location.GetGTID().Origin().Clone()) } @@ -70,7 +71,7 @@ type remoteBinlogReader struct { EnableGTID bool } -func (r *remoteBinlogReader) generateStreamer(location binlog.Location) (streamer.Streamer, error) { +func (r *remoteBinlogReader) generateStreamer(location binlog.Location) (reader.Streamer, error) { defer func() { lastSlaveConnectionID := r.reader.LastConnectionID() r.tctx.L().Info("last slave connection", zap.Uint32("connection ID", lastSlaveConnectionID)) @@ -106,7 +107,7 @@ type StreamerController struct { localBinlogDir string timezone *time.Location - streamer streamer.Streamer + streamer reader.Streamer streamerProducer StreamerProducer // meetError means meeting error when get binlog event @@ -121,11 +122,11 @@ type StreamerController struct { // whether the server id is updated serverIDUpdated bool - notifier streamer.EventNotifier + notifier relay.EventNotifier } // NewStreamerController creates a new streamer controller. -func NewStreamerController(notifier streamer.EventNotifier, +func NewStreamerController(notifier relay.EventNotifier, syncCfg replication.BinlogSyncerConfig, enableGTID bool, fromDB *dbconn.UpStreamConn, @@ -234,7 +235,7 @@ func (c *StreamerController) resetReplicationSyncer(tctx *tcontext.Context, loca if c.currentBinlogType == RemoteBinlog { c.streamerProducer = &remoteBinlogReader{replication.NewBinlogSyncer(c.syncCfg), tctx, c.syncCfg.Flavor, c.enableGTID} } else { - c.streamerProducer = &localBinlogReader{streamer.NewBinlogReader(c.notifier, tctx.L(), &streamer.BinlogReaderConfig{RelayDir: c.localBinlogDir, Timezone: c.timezone, Flavor: c.syncCfg.Flavor}), c.enableGTID} + c.streamerProducer = &localBinlogReader{relay.NewBinlogReader(c.notifier, tctx.L(), &relay.BinlogReaderConfig{RelayDir: c.localBinlogDir, Timezone: c.timezone, Flavor: c.syncCfg.Flavor}), c.enableGTID} } c.streamer, err = c.streamerProducer.generateStreamer(location) diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 1351284930b..e68d5e9b12f 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -63,6 +63,7 @@ import ( "github.com/pingcap/ticdc/dm/pkg/streamer" "github.com/pingcap/ticdc/dm/pkg/terror" "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/ticdc/dm/relay" "github.com/pingcap/ticdc/dm/syncer/dbconn" operator "github.com/pingcap/ticdc/dm/syncer/err-operator" "github.com/pingcap/ticdc/dm/syncer/metrics" @@ -223,11 +224,11 @@ type Syncer struct { workerJobTSArray []*atomic.Int64 // worker's sync job TS array, note that idx=0 is skip idx and idx=1 is ddl idx,sql worker job idx=(queue id + 2) lastCheckpointFlushedTime time.Time - notifier streamer.EventNotifier + notifier relay.EventNotifier } // NewSyncer creates a new Syncer. -func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, notifier streamer.EventNotifier) *Syncer { +func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, notifier relay.EventNotifier) *Syncer { logger := log.With(zap.String("task", cfg.Name), zap.String("unit", "binlog replication")) syncer := &Syncer{ pessimist: shardddl.NewPessimist(&logger, etcdClient, cfg.Name, cfg.SourceID), @@ -1627,7 +1628,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { return err1 } continue - case err == streamer.ErrorMaybeDuplicateEvent: + case err == relay.ErrorMaybeDuplicateEvent: tctx.L().Warn("read binlog met a truncated file, need to open safe-mode until the next transaction") err = maybeSkipNRowsEvent(eventIndex) if err == nil { diff --git a/dm/syncer/syncer_test.go b/dm/syncer/syncer_test.go index 4d1dffc8af0..65adcd893bd 100644 --- a/dm/syncer/syncer_test.go +++ b/dm/syncer/syncer_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/ticdc/dm/dm/pb" "github.com/pingcap/ticdc/dm/pkg/binlog" "github.com/pingcap/ticdc/dm/pkg/binlog/event" + "github.com/pingcap/ticdc/dm/pkg/binlog/reader" "github.com/pingcap/ticdc/dm/pkg/conn" tcontext "github.com/pingcap/ticdc/dm/pkg/context" "github.com/pingcap/ticdc/dm/pkg/cputil" @@ -37,7 +38,6 @@ import ( parserpkg "github.com/pingcap/ticdc/dm/pkg/parser" "github.com/pingcap/ticdc/dm/pkg/retry" "github.com/pingcap/ticdc/dm/pkg/schema" - streamer2 "github.com/pingcap/ticdc/dm/pkg/streamer" "github.com/pingcap/ticdc/dm/syncer/dbconn" sqlmock "github.com/DATA-DOG/go-sqlmock" @@ -112,7 +112,7 @@ type MockStreamProducer struct { events []*replication.BinlogEvent } -func (mp *MockStreamProducer) generateStreamer(location binlog.Location) (streamer2.Streamer, error) { +func (mp *MockStreamProducer) generateStreamer(location binlog.Location) (reader.Streamer, error) { if location.Position.Pos == 4 { return &MockStreamer{mp.events, 0}, nil } diff --git a/dm/tests/dmctl_basic/run.sh b/dm/tests/dmctl_basic/run.sh index 7af71ca292f..eebced09cf3 100755 --- a/dm/tests/dmctl_basic/run.sh +++ b/dm/tests/dmctl_basic/run.sh @@ -162,7 +162,7 @@ function test_operate_task_bound_to_a_source() { function run() { inject_points=( - "github.com/pingcap/ticdc/dm/pkg/streamer/SetHeartbeatInterval=return(1)" + "github.com/pingcap/ticdc/dm/relay/SetHeartbeatInterval=return(1)" ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" diff --git a/dm/tests/duplicate_event/run.sh b/dm/tests/duplicate_event/run.sh index f70b80c5805..cc8773bdf08 100644 --- a/dm/tests/duplicate_event/run.sh +++ b/dm/tests/duplicate_event/run.sh @@ -41,7 +41,7 @@ function run() { # with a 5 rows insert txn: 1 * FormatDesc + 1 * PreviousGTID + 1 * GTID + 1 * BEGIN + 5 * (Table_map + Write_rows) + 1 * XID # here we fail at the third write rows event, sync should retry and auto recover without any duplicate event - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/relay/RelayGetEventFailed=15*return(3);github.com/pingcap/ticdc/dm/relay/retry/RelayAllowRetry=return" + export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/relay/RelayGetEventFailed=15*return(3);github.com/pingcap/ticdc/dm/relay/RelayAllowRetry=return" run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT diff --git a/dm/tests/only_dml/run.sh b/dm/tests/only_dml/run.sh index 52c666b7ad0..b0e3412dab5 100755 --- a/dm/tests/only_dml/run.sh +++ b/dm/tests/only_dml/run.sh @@ -42,7 +42,7 @@ function insert_data() { } function run() { - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/pkg/streamer/SetHeartbeatInterval=return(1);github.com/pingcap/ticdc/dm/syncer/syncDMLBatchNotFull=return(true)" + export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/relay/SetHeartbeatInterval=return(1);github.com/pingcap/ticdc/dm/syncer/syncDMLBatchNotFull=return(true)" run_sql_file $cur/data/db1.prepare.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 check_contains 'Query OK, 1 row affected' diff --git a/dm/tests/relay_interrupt/run.sh b/dm/tests/relay_interrupt/run.sh index 6d9abd39c8b..bda66dc71f5 100644 --- a/dm/tests/relay_interrupt/run.sh +++ b/dm/tests/relay_interrupt/run.sh @@ -103,7 +103,7 @@ function run() { echo "read binlog from relay log failed, and will use remote binlog" kill_dm_worker - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/pkg/streamer/GetEventFromLocalFailed=return()" + export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/relay/GetEventFromLocalFailed=return()" run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT prepare_data2 $i diff --git a/dm/tests/sharding2/run.sh b/dm/tests/sharding2/run.sh index cf8c6530add..da4ab00a351 100755 --- a/dm/tests/sharding2/run.sh +++ b/dm/tests/sharding2/run.sh @@ -75,7 +75,7 @@ function run() { check_port_offline $WORKER2_PORT 20 # mock recover relay writer - export GO_FAILPOINTS='github.com/pingcap/ticdc/dm/relay/writer/MockRecoverRelayWriter=return(true)' + export GO_FAILPOINTS='github.com/pingcap/ticdc/dm/relay/MockRecoverRelayWriter=return(true)' run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT From 2137acb976838c91f97247b7820d8b5eb2595d2e Mon Sep 17 00:00:00 2001 From: Ling Jin <7138436+3AceShowHand@users.noreply.github.com> Date: Fri, 5 Nov 2021 18:29:03 +0800 Subject: [PATCH 2/5] sink: canal json support encode checkpoint event. (#3076) --- cdc/sink/codec/canal_flat.go | 173 ++++++++++++++---- cdc/sink/codec/canal_flat_test.go | 48 ++++- cdc/sink/mq.go | 24 ++- cdc/sink/mq_test.go | 4 +- cdc/sink/mysql.go | 2 +- cdc/sink/producer/kafka/kafka.go | 57 +++--- cdc/sink/producer/kafka/kafka_test.go | 5 - cdc/sink/sink.go | 2 +- .../framework/canal/kafka_single_table.go | 13 +- integration/integration.go | 18 ++ pkg/filter/filter.go | 2 +- proto/EntryProtocol.proto | 42 ++--- 12 files changed, 286 insertions(+), 104 deletions(-) diff --git a/cdc/sink/codec/canal_flat.go b/cdc/sink/codec/canal_flat.go index 5929488db07..664ffe59022 100644 --- a/cdc/sink/codec/canal_flat.go +++ b/cdc/sink/codec/canal_flat.go @@ -16,6 +16,8 @@ package codec import ( "context" "encoding/json" + "strconv" + "time" "github.com/pingcap/errors" "github.com/pingcap/log" @@ -28,16 +30,22 @@ import ( // CanalFlatEventBatchEncoder encodes Canal flat messages in JSON format type CanalFlatEventBatchEncoder struct { builder *canalEntryBuilder - unresolvedBuf []*canalFlatMessage - resolvedBuf []*canalFlatMessage + unresolvedBuf []canalFlatMessageInterface + resolvedBuf []canalFlatMessageInterface + // When it is true, canal-json would generate TiDB extension information + // which, at the moment, only includes `tidbWaterMarkType` and `_tidb` fields. + enableTiDBExtension bool } +const tidbWaterMarkType = "TIDB_WATERMARK" + // NewCanalFlatEventBatchEncoder creates a new CanalFlatEventBatchEncoder func NewCanalFlatEventBatchEncoder() EventBatchEncoder { return &CanalFlatEventBatchEncoder{ - builder: NewCanalEntryBuilder(), - unresolvedBuf: make([]*canalFlatMessage, 0), - resolvedBuf: make([]*canalFlatMessage, 0), + builder: NewCanalEntryBuilder(), + unresolvedBuf: make([]canalFlatMessageInterface, 0), + resolvedBuf: make([]canalFlatMessageInterface, 0), + enableTiDBExtension: false, } } @@ -59,20 +67,32 @@ func newCanalFlatEventBatchEncoderBuilder(opts map[string]string) EncoderBuilder return &canalFlatEventBatchEncoderBuilder{opts: opts} } +// The TiCDC Canal-JSON implementation extend the official format with a TiDB extension field. +// canalFlatMessageInterface is used to support this without affect the original format. +type canalFlatMessageInterface interface { + getTikvTs() uint64 + getSchema() *string + getTable() *string +} + // adapted from https://github.com/alibaba/canal/blob/master/protocol/src/main/java/com/alibaba/otter/canal/protocol/FlatMessage.java type canalFlatMessage struct { // ignored by consumers - ID int64 `json:"id"` - Schema string `json:"database"` - Table string `json:"table"` - PKNames []string `json:"pkNames"` - IsDDL bool `json:"isDdl"` - EventType string `json:"type"` - ExecutionTime int64 `json:"es"` - // officially the time of building the MQ message, actually ignored - BuildTime uint64 `json:"ts"` - Query string `json:"sql"` - SQLType map[string]int32 `json:"sqlType"` + ID int64 `json:"id"` + Schema string `json:"database"` + Table string `json:"table"` + PKNames []string `json:"pkNames"` + IsDDL bool `json:"isDdl"` + EventType string `json:"type"` + // officially the timestamp of the event-time of the message, in milliseconds since Epoch. + ExecutionTime int64 `json:"es"` + // officially the timestamp of building the MQ message, in milliseconds since Epoch. + BuildTime int64 `json:"ts"` + // SQL that generated the change event, DDL or Query + Query string `json:"sql"` + // only works for INSERT / UPDATE / DELETE events, records each column's java representation type. + SQLType map[string]int32 `json:"sqlType"` + // only works for INSERT / UPDATE / DELETE events, records each column's mysql representation type. MySQLType map[string]string `json:"mysqlType"` // A Datum should be a string or nil Data []map[string]interface{} `json:"data"` @@ -81,7 +101,45 @@ type canalFlatMessage struct { tikvTs uint64 } -func (c *CanalFlatEventBatchEncoder) newFlatMessageForDML(e *model.RowChangedEvent) (*canalFlatMessage, error) { +func (c *canalFlatMessage) getTikvTs() uint64 { + return c.tikvTs +} + +func (c *canalFlatMessage) getSchema() *string { + return &c.Schema +} + +func (c *canalFlatMessage) getTable() *string { + return &c.Table +} + +type tidbExtension struct { + CommitTs uint64 `json:"commit-ts"` + WatermarkTs uint64 `json:"watermark-ts"` +} + +type canalFlatMessageWithTiDBExtension struct { + *canalFlatMessage + // Extensions is a TiCDC custom field that different from official Canal-JSON format. + // It would be useful to store something for special usage. + // At the moment, only store the `tso` of each event, + // which is useful if the message consumer needs to restore the original transactions. + Extensions *tidbExtension `json:"_tidb"` +} + +func (c *canalFlatMessageWithTiDBExtension) getTikvTs() uint64 { + return c.tikvTs +} + +func (c *canalFlatMessageWithTiDBExtension) getSchema() *string { + return &c.Schema +} + +func (c *canalFlatMessageWithTiDBExtension) getTable() *string { + return &c.Table +} + +func (c *CanalFlatEventBatchEncoder) newFlatMessageForDML(e *model.RowChangedEvent) (canalFlatMessageInterface, error) { eventType := convertRowEventType(e) header := c.builder.buildHeader(e.CommitTs, e.Table.Schema, e.Table.Table, eventType, 1) rowData, err := c.builder.buildRowData(e) @@ -140,7 +198,7 @@ func (c *CanalFlatEventBatchEncoder) newFlatMessageForDML(e *model.RowChangedEve data = oldData } - ret := &canalFlatMessage{ + flatMessage := &canalFlatMessage{ ID: 0, // ignored by both Canal Adapter and Flink Schema: header.SchemaName, Table: header.TableName, @@ -148,7 +206,7 @@ func (c *CanalFlatEventBatchEncoder) newFlatMessageForDML(e *model.RowChangedEve IsDDL: false, EventType: header.GetEventType().String(), ExecutionTime: header.ExecuteTime, - BuildTime: 0, // ignored by both Canal Adapter and Flink + BuildTime: time.Now().UnixNano() / 1e6, // ignored by both Canal Adapter and Flink Query: "", SQLType: sqlType, MySQLType: mysqlType, @@ -159,41 +217,77 @@ func (c *CanalFlatEventBatchEncoder) newFlatMessageForDML(e *model.RowChangedEve // We need to ensure that both Data and Old have exactly one element, // even if the element could be nil. Changing this could break Alibaba's adapter - ret.Data = append(ret.Data, data) - ret.Old = append(ret.Old, oldData) + flatMessage.Data = append(flatMessage.Data, data) + flatMessage.Old = append(flatMessage.Old, oldData) + + if !c.enableTiDBExtension { + return flatMessage, nil + } - return ret, nil + return &canalFlatMessageWithTiDBExtension{ + canalFlatMessage: flatMessage, + Extensions: &tidbExtension{CommitTs: e.CommitTs}, + }, nil } -func (c *CanalFlatEventBatchEncoder) newFlatMessageForDDL(e *model.DDLEvent) *canalFlatMessage { +func (c *CanalFlatEventBatchEncoder) newFlatMessageForDDL(e *model.DDLEvent) canalFlatMessageInterface { header := c.builder.buildHeader(e.CommitTs, e.TableInfo.Schema, e.TableInfo.Table, convertDdlEventType(e), 1) - - ret := &canalFlatMessage{ + flatMessage := &canalFlatMessage{ ID: 0, // ignored by both Canal Adapter and Flink Schema: header.SchemaName, Table: header.TableName, IsDDL: true, EventType: header.GetEventType().String(), ExecutionTime: header.ExecuteTime, - BuildTime: 0, // ignored by both Canal Adapter and Flink + BuildTime: time.Now().UnixNano() / 1e6, // timestamp Query: e.Query, tikvTs: e.CommitTs, } - return ret + + if !c.enableTiDBExtension { + return flatMessage + } + + return &canalFlatMessageWithTiDBExtension{ + canalFlatMessage: flatMessage, + Extensions: &tidbExtension{CommitTs: e.CommitTs}, + } +} + +func (c *CanalFlatEventBatchEncoder) newFlatMessage4CheckpointEvent(ts uint64) *canalFlatMessageWithTiDBExtension { + return &canalFlatMessageWithTiDBExtension{ + canalFlatMessage: &canalFlatMessage{ + ID: 0, + IsDDL: false, + EventType: tidbWaterMarkType, + ExecutionTime: convertToCanalTs(ts), + BuildTime: time.Now().UnixNano() / int64(time.Millisecond), // converts to milliseconds + }, + Extensions: &tidbExtension{WatermarkTs: ts}, + } } -// EncodeCheckpointEvent is no-op +// EncodeCheckpointEvent implements the EventBatchEncoder interface func (c *CanalFlatEventBatchEncoder) EncodeCheckpointEvent(ts uint64) (*MQMessage, error) { - return nil, nil + if !c.enableTiDBExtension { + return nil, nil + } + + msg := c.newFlatMessage4CheckpointEvent(ts) + value, err := json.Marshal(msg) + if err != nil { + return nil, cerrors.WrapError(cerrors.ErrCanalEncodeFailed, err) + } + return newResolvedMQMessage(ProtocolCanalJSON, nil, value, ts), nil } // AppendRowChangedEvent implements the interface EventBatchEncoder func (c *CanalFlatEventBatchEncoder) AppendRowChangedEvent(e *model.RowChangedEvent) (EncoderResult, error) { - msg, err := c.newFlatMessageForDML(e) + message, err := c.newFlatMessageForDML(e) if err != nil { return EncoderNoOperation, errors.Trace(err) } - c.unresolvedBuf = append(c.unresolvedBuf, msg) + c.unresolvedBuf = append(c.unresolvedBuf, message) return EncoderNoOperation, nil } @@ -201,7 +295,7 @@ func (c *CanalFlatEventBatchEncoder) AppendRowChangedEvent(e *model.RowChangedEv func (c *CanalFlatEventBatchEncoder) AppendResolvedEvent(ts uint64) (EncoderResult, error) { nextIdx := 0 for _, msg := range c.unresolvedBuf { - if msg.tikvTs <= ts { + if msg.getTikvTs() <= ts { c.resolvedBuf = append(c.resolvedBuf, msg) } else { break @@ -217,8 +311,8 @@ func (c *CanalFlatEventBatchEncoder) AppendResolvedEvent(ts uint64) (EncoderResu // EncodeDDLEvent encodes DDL events func (c *CanalFlatEventBatchEncoder) EncodeDDLEvent(e *model.DDLEvent) (*MQMessage, error) { - msg := c.newFlatMessageForDDL(e) - value, err := json.Marshal(msg) + message := c.newFlatMessageForDDL(e) + value, err := json.Marshal(message) if err != nil { return nil, cerrors.WrapError(cerrors.ErrCanalEncodeFailed, err) } @@ -237,7 +331,7 @@ func (c *CanalFlatEventBatchEncoder) Build() []*MQMessage { log.Panic("CanalFlatEventBatchEncoder", zap.Error(err)) return nil } - ret[i] = NewMQMessage(ProtocolCanalJSON, nil, value, msg.tikvTs, model.MqMessageTypeRow, &msg.Schema, &msg.Table) + ret[i] = NewMQMessage(ProtocolCanalJSON, nil, value, msg.getTikvTs(), model.MqMessageTypeRow, msg.getSchema(), msg.getTable()) } c.resolvedBuf = c.resolvedBuf[0:0] return ret @@ -258,8 +352,13 @@ func (c *CanalFlatEventBatchEncoder) Reset() { panic("not supported") } -// SetParams is no-op for now func (c *CanalFlatEventBatchEncoder) SetParams(params map[string]string) error { - // no op + if s, ok := params["enable-tidb-extension"]; ok { + a, err := strconv.ParseBool(s) + if err != nil { + return cerrors.WrapError(cerrors.ErrSinkInvalidConfig, err) + } + c.enableTiDBExtension = a + } return nil } diff --git a/cdc/sink/codec/canal_flat_test.go b/cdc/sink/codec/canal_flat_test.go index 14df059e673..f031fc7ff32 100644 --- a/cdc/sink/codec/canal_flat_test.go +++ b/cdc/sink/codec/canal_flat_test.go @@ -32,9 +32,11 @@ func (s *canalFlatSuite) TestNewCanalFlatMessageFromDML(c *check.C) { defer testleak.AfterTest(c)() encoder := &CanalFlatEventBatchEncoder{builder: NewCanalEntryBuilder()} c.Assert(encoder, check.NotNil) - msg, err := encoder.newFlatMessageForDML(testCaseUpdate) + message, err := encoder.newFlatMessageForDML(testCaseUpdate) c.Assert(err, check.IsNil) + msg, ok := message.(*canalFlatMessage) + c.Assert(ok, check.IsTrue) c.Assert(msg.EventType, check.Equals, "UPDATE") c.Assert(msg.ExecutionTime, check.Equals, convertToCanalTs(testCaseUpdate.CommitTs)) c.Assert(msg.tikvTs, check.Equals, testCaseUpdate.CommitTs) @@ -75,6 +77,17 @@ func (s *canalFlatSuite) TestNewCanalFlatMessageFromDML(c *check.C) { "blob": string(encodedBytes), }, }) + + encoder = &CanalFlatEventBatchEncoder{builder: NewCanalEntryBuilder(), enableTiDBExtension: true} + c.Assert(encoder, check.NotNil) + message, err = encoder.newFlatMessageForDML(testCaseUpdate) + c.Assert(err, check.IsNil) + + withExtension, ok := message.(*canalFlatMessageWithTiDBExtension) + c.Assert(ok, check.IsTrue) + + c.Assert(withExtension.Extensions, check.NotNil) + c.Assert(withExtension.Extensions.CommitTs, check.Equals, testCaseUpdate.CommitTs) } func (s *canalFlatSuite) TestNewCanalFlatMessageFromDDL(c *check.C) { @@ -82,9 +95,11 @@ func (s *canalFlatSuite) TestNewCanalFlatMessageFromDDL(c *check.C) { encoder := &CanalFlatEventBatchEncoder{builder: NewCanalEntryBuilder()} c.Assert(encoder, check.NotNil) - msg := encoder.newFlatMessageForDDL(testCaseDdl) - c.Assert(msg, check.NotNil) + message := encoder.newFlatMessageForDDL(testCaseDdl) + c.Assert(message, check.NotNil) + msg, ok := message.(*canalFlatMessage) + c.Assert(ok, check.IsTrue) c.Assert(msg.tikvTs, check.Equals, testCaseDdl.CommitTs) c.Assert(msg.ExecutionTime, check.Equals, convertToCanalTs(testCaseDdl.CommitTs)) c.Assert(msg.IsDDL, check.IsTrue) @@ -92,6 +107,18 @@ func (s *canalFlatSuite) TestNewCanalFlatMessageFromDDL(c *check.C) { c.Assert(msg.Table, check.Equals, "person") c.Assert(msg.Query, check.Equals, testCaseDdl.Query) c.Assert(msg.EventType, check.Equals, "CREATE") + + encoder = &CanalFlatEventBatchEncoder{builder: NewCanalEntryBuilder(), enableTiDBExtension: true} + c.Assert(encoder, check.NotNil) + + message = encoder.newFlatMessageForDDL(testCaseDdl) + c.Assert(message, check.NotNil) + + withExtension, ok := message.(*canalFlatMessageWithTiDBExtension) + c.Assert(ok, check.IsTrue) + + c.Assert(withExtension.Extensions, check.NotNil) + c.Assert(withExtension.Extensions.CommitTs, check.Equals, testCaseDdl.CommitTs) } func (s *canalFlatSuite) TestBatching(c *check.C) { @@ -138,6 +165,21 @@ func (s *canalFlatSuite) TestBatching(c *check.C) { c.Assert(encoder.resolvedBuf, check.HasLen, 0) } +func (s *canalFlatSuite) TestEncodeCheckpointEvent(c *check.C) { + defer testleak.AfterTest(c)() + encoder := &CanalFlatEventBatchEncoder{builder: NewCanalEntryBuilder(), enableTiDBExtension: false} + c.Assert(encoder, check.NotNil) + + msg, err := encoder.EncodeCheckpointEvent(2333) + c.Assert(err, check.IsNil) + c.Assert(msg, check.IsNil) + + encoder.enableTiDBExtension = true + msg, err = encoder.EncodeCheckpointEvent(2333) + c.Assert(err, check.IsNil) + c.Assert(msg, check.NotNil) +} + var testCaseUpdate = &model.RowChangedEvent{ CommitTs: 417318403368288260, Table: &model.TableName{ diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index 357eb82b507..0b39db09c60 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -42,7 +42,11 @@ type mqEvent struct { resolvedTs uint64 } -const defaultPartitionInputChSize = 12800 +const ( + defaultPartitionInputChSize = 12800 + // -1 means broadcast to all partitions, it's the default for the default open protocol. + defaultDDLDispatchPartition = -1 +) type mqSink struct { mqProducer producer.Producer @@ -230,9 +234,18 @@ func (k *mqSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { return nil } + var partition int32 = defaultDDLDispatchPartition + // for Canal-JSON / Canal-PB, send to partition 0. + if _, ok := encoder.(*codec.CanalFlatEventBatchEncoder); ok { + partition = 0 + } + if _, ok := encoder.(*codec.CanalEventBatchEncoder); ok { + partition = 0 + } + k.statistics.AddDDLCount() - log.Debug("emit ddl event", zap.String("query", ddl.Query), zap.Uint64("commit-ts", ddl.CommitTs)) - err = k.writeToProducer(ctx, msg, codec.EncoderNeedSyncWrite, -1) + log.Debug("emit ddl event", zap.String("query", ddl.Query), zap.Uint64("commit-ts", ddl.CommitTs), zap.Int32("partition", partition)) + err = k.writeToProducer(ctx, msg, codec.EncoderNeedSyncWrite, partition) return errors.Trace(err) } @@ -372,11 +385,6 @@ func (k *mqSink) writeToProducer(ctx context.Context, message *codec.MQMessage, } func newKafkaSaramaSink(ctx context.Context, sinkURI *url.URL, filter *filter.Filter, replicaConfig *config.ReplicaConfig, opts map[string]string, errCh chan error) (*mqSink, error) { - scheme := strings.ToLower(sinkURI.Scheme) - if scheme != "kafka" && scheme != "kafka+ssl" { - return nil, cerror.ErrKafkaInvalidConfig.GenWithStack("can't create MQ sink with unsupported scheme: %s", scheme) - } - config := kafka.NewConfig() if err := config.Initialize(sinkURI, replicaConfig, opts); err != nil { return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, err) diff --git a/cdc/sink/mq_test.go b/cdc/sink/mq_test.go index b3f80867a59..ee5b0146f60 100644 --- a/cdc/sink/mq_test.go +++ b/cdc/sink/mq_test.go @@ -53,7 +53,7 @@ func (s mqSinkSuite) TestKafkaSink(c *check.C) { uriTemplate := "kafka://%s/kafka-test?kafka-version=0.9.0.0&max-batch-size=1" + "&max-message-bytes=4194304&partition-num=1" + - "&kafka-client-id=unit-test&auto-create-topic=false&compression=gzip" + "&kafka-client-id=unit-test&auto-create-topic=false&compression=gzip&protocol=default" uri := fmt.Sprintf(uriTemplate, leader.Addr()) sinkURI, err := url.Parse(uri) c.Assert(err, check.IsNil) @@ -149,7 +149,7 @@ func (s mqSinkSuite) TestKafkaSinkFilter(c *check.C) { prodSuccess := new(sarama.ProduceResponse) prodSuccess.AddTopicPartition(topic, 0, sarama.ErrNoError) - uriTemplate := "kafka://%s/kafka-test?kafka-version=0.9.0.0&auto-create-topic=false" + uriTemplate := "kafka://%s/kafka-test?kafka-version=0.9.0.0&auto-create-topic=false&protocol=default" uri := fmt.Sprintf(uriTemplate, leader.Addr()) sinkURI, err := url.Parse(uri) c.Assert(err, check.IsNil) diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index b74efd48736..9e66b57f5ad 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -120,7 +120,7 @@ func (s *mysqlSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.Row return nil } -// FlushRowChangedEvents will flushes all received events, we don't allow mysql +// FlushRowChangedEvents will flush all received events, we don't allow mysql // sink to receive events before resolving func (s *mysqlSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { if atomic.LoadUint64(&s.maxResolvedTs) < resolvedTs { diff --git a/cdc/sink/producer/kafka/kafka.go b/cdc/sink/producer/kafka/kafka.go index 5f51e495565..8f3a6a6585d 100644 --- a/cdc/sink/producer/kafka/kafka.go +++ b/cdc/sink/producer/kafka/kafka.go @@ -70,16 +70,20 @@ func NewConfig() *Config { // Initialize the kafka configuration func (c *Config) Initialize(sinkURI *url.URL, replicaConfig *config.ReplicaConfig, opts map[string]string) error { - s := sinkURI.Query().Get("partition-num") + params := sinkURI.Query() + s := params.Get("partition-num") if s != "" { a, err := strconv.Atoi(s) if err != nil { return err } c.PartitionNum = int32(a) + if c.PartitionNum <= 0 { + return cerror.ErrKafkaInvalidPartitionNum.GenWithStackByArgs(c.PartitionNum) + } } - s = sinkURI.Query().Get("replication-factor") + s = params.Get("replication-factor") if s != "" { a, err := strconv.Atoi(s) if err != nil { @@ -88,12 +92,12 @@ func (c *Config) Initialize(sinkURI *url.URL, replicaConfig *config.ReplicaConfi c.ReplicationFactor = int16(a) } - s = sinkURI.Query().Get("kafka-version") + s = params.Get("kafka-version") if s != "" { c.Version = s } - s = sinkURI.Query().Get("max-message-bytes") + s = params.Get("max-message-bytes") if s != "" { a, err := strconv.Atoi(s) if err != nil { @@ -103,54 +107,49 @@ func (c *Config) Initialize(sinkURI *url.URL, replicaConfig *config.ReplicaConfi opts["max-message-bytes"] = s } - s = sinkURI.Query().Get("max-batch-size") + s = params.Get("max-batch-size") if s != "" { opts["max-batch-size"] = s } - s = sinkURI.Query().Get("compression") + s = params.Get("compression") if s != "" { c.Compression = s } - c.ClientID = sinkURI.Query().Get("kafka-client-id") - - s = sinkURI.Query().Get("protocol") - if s != "" { - replicaConfig.Sink.Protocol = s - } + c.ClientID = params.Get("kafka-client-id") - s = sinkURI.Query().Get("ca") + s = params.Get("ca") if s != "" { c.Credential.CAPath = s } - s = sinkURI.Query().Get("cert") + s = params.Get("cert") if s != "" { c.Credential.CertPath = s } - s = sinkURI.Query().Get("key") + s = params.Get("key") if s != "" { c.Credential.KeyPath = s } - s = sinkURI.Query().Get("sasl-user") + s = params.Get("sasl-user") if s != "" { c.SaslScram.SaslUser = s } - s = sinkURI.Query().Get("sasl-password") + s = params.Get("sasl-password") if s != "" { c.SaslScram.SaslPassword = s } - s = sinkURI.Query().Get("sasl-mechanism") + s = params.Get("sasl-mechanism") if s != "" { c.SaslScram.SaslMechanism = s } - s = sinkURI.Query().Get("auto-create-topic") + s = params.Get("auto-create-topic") if s != "" { autoCreate, err := strconv.ParseBool(s) if err != nil { @@ -159,6 +158,23 @@ func (c *Config) Initialize(sinkURI *url.URL, replicaConfig *config.ReplicaConfi c.TopicPreProcess = autoCreate } + s = params.Get("protocol") + if s != "" { + replicaConfig.Sink.Protocol = s + } + + s = params.Get("enable-tidb-extension") + if s != "" { + _, err := strconv.ParseBool(s) + if err != nil { + return err + } + if replicaConfig.Sink.Protocol != "canal-json" { + return cerror.WrapError(cerror.ErrKafkaInvalidConfig, errors.New("enable-tidb-extension only support canal-json")) + } + opts["enable-tidb-extension"] = s + } + return nil } @@ -438,9 +454,6 @@ func NewKafkaSaramaProducer(ctx context.Context, address string, topic string, c if err != nil { return nil, err } - if config.PartitionNum < 0 { - return nil, cerror.ErrKafkaInvalidPartitionNum.GenWithStackByArgs(config.PartitionNum) - } asyncClient, err := sarama.NewAsyncProducer(strings.Split(address, ","), cfg) if err != nil { return nil, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) diff --git a/cdc/sink/producer/kafka/kafka_test.go b/cdc/sink/producer/kafka/kafka_test.go index b54965e8cad..f44fa8ae8ee 100644 --- a/cdc/sink/producer/kafka/kafka_test.go +++ b/cdc/sink/producer/kafka/kafka_test.go @@ -347,11 +347,6 @@ func (s *kafkaSuite) TestCreateProducerFailed(c *check.C) { config.Version = "invalid" _, err := NewKafkaSaramaProducer(ctx, "127.0.0.1:1111", "topic", config, errCh) c.Assert(errors.Cause(err), check.ErrorMatches, "invalid version.*") - - config.Version = "0.8.2.0" - config.PartitionNum = int32(-1) - _, err = NewKafkaSaramaProducer(ctx, "127.0.0.1:1111", "topic", config, errCh) - c.Assert(cerror.ErrKafkaInvalidPartitionNum.Equal(err), check.IsTrue) } func (s *kafkaSuite) TestProducerSendMessageFailed(c *check.C) { diff --git a/cdc/sink/sink.go b/cdc/sink/sink.go index c5d7cd18ca0..3d28595a707 100644 --- a/cdc/sink/sink.go +++ b/cdc/sink/sink.go @@ -44,7 +44,7 @@ type Sink interface { EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error // FlushRowChangedEvents flushes each row which of commitTs less than or equal to `resolvedTs` into downstream. - // TiCDC guarantees that all of Event which of commitTs less than or equal to `resolvedTs` are sent to Sink through `EmitRowChangedEvents` + // TiCDC guarantees that all the Events whose commitTs is less than or equal to `resolvedTs` are sent to Sink through `EmitRowChangedEvents` FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) // EmitCheckpointTs sends CheckpointTs to Sink diff --git a/integration/framework/canal/kafka_single_table.go b/integration/framework/canal/kafka_single_table.go index 75155348995..d925f062dbf 100644 --- a/integration/framework/canal/kafka_single_table.go +++ b/integration/framework/canal/kafka_single_table.go @@ -27,8 +27,9 @@ const ( // SingleTableTask provides a basic implementation for an Avro test case type SingleTableTask struct { - TableName string - UseJSON bool + TableName string + UseJSON bool + EnableTiDBExtension bool } // Name implements Task @@ -45,9 +46,15 @@ func (c *SingleTableTask) GetCDCProfile() *framework.CDCProfile { } else { protocol = "canal" } + + sinkURI := "kafka://kafka:9092/" + testDbName + "?kafka-version=2.6.0&protocol=" + protocol + if c.EnableTiDBExtension { + sinkURI += "&enable-tidb-extension=true" + } + return &framework.CDCProfile{ PDUri: "http://upstream-pd:2379", - SinkURI: "kafka://kafka:9092/" + testDbName + "?kafka-version=2.6.0&protocol=" + protocol, + SinkURI: sinkURI, Opts: map[string]string{"force-handle-key-pkey": "true", "support-txn": "true"}, ConfigFile: "/configs/canal-test-config.toml", } diff --git a/integration/integration.go b/integration/integration.go index e77d59df08e..27c27d48c89 100644 --- a/integration/integration.go +++ b/integration/integration.go @@ -80,6 +80,22 @@ func testCanalJSON() { runTests(testCases, env) } +func testCanalJSONWatermark() { + env := canal.NewKafkaDockerEnv(*dockerComposeFile) + env.DockerComposeOperator.ExecEnv = []string{"USE_FLAT_MESSAGE=true"} + task := &canal.SingleTableTask{TableName: "test", UseJSON: true, EnableTiDBExtension: true} + testCases := []framework.Task{ + tests.NewSimpleCase(task), + tests.NewDeleteCase(task), + tests.NewManyTypesCase(task), + // tests.NewUnsignedCase(task), //now canal adapter can not deal with unsigned int greater than int max + tests.NewCompositePKeyCase(task), + tests.NewAlterCase(task), + } + + runTests(testCases, env) +} + func testMySQL() { env := mysql.NewDockerEnv(*dockerComposeFile) task := &mysql.SingleTableTask{TableName: "test"} @@ -133,6 +149,8 @@ func main() { testCanal() } else if *testProtocol == "canalJson" { testCanalJSON() + } else if *testProtocol == "canalJson-extension" { + testCanalJSONWatermark() } else if *testProtocol == "mysql" { testMySQL() } else if *testProtocol == "simple-mysql-checking-old-value" { diff --git a/pkg/filter/filter.go b/pkg/filter/filter.go index afe5913c5cf..596e1f06094 100644 --- a/pkg/filter/filter.go +++ b/pkg/filter/filter.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/tidb/parser/model" ) -// Filter is a event filter implementation. +// Filter is an event filter implementation. type Filter struct { filter filterV2.Filter ignoreTxnStartTs []uint64 diff --git a/proto/EntryProtocol.proto b/proto/EntryProtocol.proto index 1915cdccd4c..b7c382babc9 100644 --- a/proto/EntryProtocol.proto +++ b/proto/EntryProtocol.proto @@ -217,30 +217,30 @@ enum EntryType{ /** 事件类型 **/ enum EventType { EVENTTYPECOMPATIBLEPROTO2 = 0; - INSERT = 1; - UPDATE = 2; - DELETE = 3; - CREATE = 4; - ALTER = 5; - ERASE = 6; - QUERY = 7; - TRUNCATE = 8; - RENAME = 9; - /**CREATE INDEX**/ - CINDEX = 10; - DINDEX = 11; - GTID = 12; - /** XA **/ - XACOMMIT = 13; - XAROLLBACK = 14; - /** MASTER HEARTBEAT **/ - MHEARTBEAT = 15; + INSERT = 1; + UPDATE = 2; + DELETE = 3; + CREATE = 4; + ALTER = 5; + ERASE = 6; + QUERY = 7; + TRUNCATE = 8; + RENAME = 9; + /**CREATE INDEX**/ + CINDEX = 10; + DINDEX = 11; + GTID = 12; + /** XA **/ + XACOMMIT = 13; + XAROLLBACK = 14; + /** MASTER HEARTBEAT **/ + MHEARTBEAT = 15; } /**数据库类型**/ enum Type { TYPECOMPATIBLEPROTO2 = 0; - ORACLE = 1; - MYSQL = 2; - PGSQL = 3; + ORACLE = 1; + MYSQL = 2; + PGSQL = 3; } From 28424edb959bc171cb97e80fa928140619e1b4e8 Mon Sep 17 00:00:00 2001 From: dongmen <20351731+asddongmen@users.noreply.github.com> Date: Fri, 5 Nov 2021 18:53:03 +0800 Subject: [PATCH 3/5] http_*: add log for http api and refine the err handle logic (#2997) --- cdc/capture/http_errors.go | 51 +++++++++++ cdc/capture/http_errors_test.go | 33 +++++++ cdc/capture/http_handler.go | 150 ++++++++++---------------------- cdc/capture/http_validator.go | 2 +- cdc/http_router.go | 56 +++++++++++- cdc/http_router_test.go | 3 +- cdc/http_status.go | 2 +- cdc/sink/mysql.go | 24 +++-- cdc/sink/simple_mysql_tester.go | 2 +- cdc/sink/sink_test.go | 2 +- pkg/config/config_test.go | 1 + 11 files changed, 203 insertions(+), 123 deletions(-) create mode 100644 cdc/capture/http_errors.go create mode 100644 cdc/capture/http_errors_test.go diff --git a/cdc/capture/http_errors.go b/cdc/capture/http_errors.go new file mode 100644 index 00000000000..9fbe07fb4ff --- /dev/null +++ b/cdc/capture/http_errors.go @@ -0,0 +1,51 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package capture + +import ( + "strings" + + "github.com/pingcap/errors" + cerror "github.com/pingcap/ticdc/pkg/errors" +) + +// httpBadRequestError is some errors that will cause a BadRequestError in http handler +var httpBadRequestError = []*errors.Error{ + cerror.ErrAPIInvalidParam, cerror.ErrSinkURIInvalid, cerror.ErrStartTsBeforeGC, + cerror.ErrChangeFeedNotExists, cerror.ErrTargetTsBeforeStartTs, cerror.ErrTableIneligible, + cerror.ErrFilterRuleInvalid, cerror.ErrChangefeedUpdateRefused, cerror.ErrMySQLConnectionError, + cerror.ErrMySQLInvalidConfig, +} + +// IsHTTPBadRequestError check if a error is a http bad request error +func IsHTTPBadRequestError(err error) bool { + if err == nil { + return false + } + for _, e := range httpBadRequestError { + if e.Equal(err) { + return true + } + + rfcCode, ok := cerror.RFCCode(err) + if ok && e.RFCCode() == rfcCode { + return true + } + + if strings.Contains(err.Error(), string(e.RFCCode())) { + return true + } + } + return false +} diff --git a/cdc/capture/http_errors_test.go b/cdc/capture/http_errors_test.go new file mode 100644 index 00000000000..cd9592e7da8 --- /dev/null +++ b/cdc/capture/http_errors_test.go @@ -0,0 +1,33 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package capture + +import ( + "testing" + + "github.com/pingcap/errors" + cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/stretchr/testify/require" +) + +func TestIsHTTPBadRequestError(t *testing.T) { + err := cerror.ErrAPIInvalidParam.GenWithStack("aa") + require.Equal(t, true, IsHTTPBadRequestError(err)) + err = cerror.ErrAPIInvalidParam.Wrap(errors.New("aa")) + require.Equal(t, true, IsHTTPBadRequestError(err)) + err = cerror.ErrPDEtcdAPIError.GenWithStack("aa") + require.Equal(t, false, IsHTTPBadRequestError(err)) + err = nil + require.Equal(t, false, IsHTTPBadRequestError(err)) +} diff --git a/cdc/capture/http_handler.go b/cdc/capture/http_handler.go index ae937f15f0a..2b6ec2325b2 100644 --- a/cdc/capture/http_handler.go +++ b/cdc/capture/http_handler.go @@ -78,13 +78,15 @@ func (h *HTTPHandler) ListChangefeed(c *gin.Context) { // get all changefeed status statuses, err := statusProvider.GetAllChangeFeedStatuses(ctx) if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } // get all changefeed infos infos, err := statusProvider.GetAllChangeFeedInfo(ctx) if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + // this call will return a parsedError generated by the error we passed in + // so it is no need to check the parsedError + _ = c.Error(err) return } @@ -140,38 +142,25 @@ func (h *HTTPHandler) GetChangefeed(c *gin.Context) { ctx := c.Request.Context() changefeedID := c.Param(apiOpVarChangefeedID) if err := model.ValidateChangefeedID(changefeedID); err != nil { - c.IndentedJSON(http.StatusBadRequest, - model.NewHTTPError(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID))) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID)) return } info, err := statusProvider.GetChangeFeedInfo(ctx, changefeedID) if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - return - } - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } status, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - return - } - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } processorInfos, err := statusProvider.GetAllTaskStatuses(ctx, changefeedID) if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - return - } - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -197,7 +186,7 @@ func (h *HTTPHandler) GetChangefeed(c *gin.Context) { TaskStatus: taskStatus, } - c.JSON(http.StatusOK, changefeedDetail) + c.IndentedJSON(http.StatusOK, changefeedDetail) } // CreateChangefeed creates a changefeed @@ -219,29 +208,25 @@ func (h *HTTPHandler) CreateChangefeed(c *gin.Context) { ctx := c.Request.Context() var changefeedConfig model.ChangefeedConfig if err := c.BindJSON(&changefeedConfig); err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(cerror.ErrAPIInvalidParam.Wrap(err)) return } info, err := verifyCreateChangefeedConfig(c, changefeedConfig, h.capture) if err != nil { - if cerror.ErrPDEtcdAPIError.Equal(err) { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) - return - } - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) + _ = c.Error(err) return } infoStr, err := info.Marshal() if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } err = h.capture.etcdClient.CreateChangefeedInfo(ctx, info, changefeedConfig.ID) if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -269,18 +254,13 @@ func (h *HTTPHandler) PauseChangefeed(c *gin.Context) { changefeedID := c.Param(apiOpVarChangefeedID) if err := model.ValidateChangefeedID(changefeedID); err != nil { - c.IndentedJSON(http.StatusBadRequest, - model.NewHTTPError(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID))) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID)) return } // check if the changefeed exists _, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - return - } - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -316,18 +296,13 @@ func (h *HTTPHandler) ResumeChangefeed(c *gin.Context) { ctx := c.Request.Context() changefeedID := c.Param(apiOpVarChangefeedID) if err := model.ValidateChangefeedID(changefeedID); err != nil { - c.IndentedJSON(http.StatusBadRequest, - model.NewHTTPError(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID))) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID)) return } // check if the changefeed exists _, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - return - } - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -370,21 +345,16 @@ func (h *HTTPHandler) UpdateChangefeed(c *gin.Context) { changefeedID := c.Param(apiOpVarChangefeedID) if err := model.ValidateChangefeedID(changefeedID); err != nil { - c.IndentedJSON(http.StatusBadRequest, - model.NewHTTPError(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID))) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID)) return } info, err := statusProvider.GetChangeFeedInfo(ctx, changefeedID) if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - return - } - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } if info.State != model.StateStopped { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(cerror.ErrChangefeedUpdateRefused.GenWithStackByArgs("can only update changefeed config when it is stopped"))) + _ = c.Error(cerror.ErrChangefeedUpdateRefused.GenWithStackByArgs("can only update changefeed config when it is stopped")) return } @@ -392,19 +362,19 @@ func (h *HTTPHandler) UpdateChangefeed(c *gin.Context) { // filter_rules, ignore_txn_start_ts, mounter_worker_num, sink_config var changefeedConfig model.ChangefeedConfig if err = c.BindJSON(&changefeedConfig); err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } newInfo, err := verifyUpdateChangefeedConfig(ctx, changefeedConfig, info) if err != nil { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) + _ = c.Error(err) return } err = h.capture.etcdClient.SaveChangeFeedInfo(ctx, newInfo, changefeedID) if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -430,18 +400,13 @@ func (h *HTTPHandler) RemoveChangefeed(c *gin.Context) { ctx := c.Request.Context() changefeedID := c.Param(apiOpVarChangefeedID) if err := model.ValidateChangefeedID(changefeedID); err != nil { - c.IndentedJSON(http.StatusBadRequest, - model.NewHTTPError(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID))) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID)) return } // check if the changefeed exists _, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - return - } - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -478,18 +443,13 @@ func (h *HTTPHandler) RebalanceTable(c *gin.Context) { changefeedID := c.Param(apiOpVarChangefeedID) if err := model.ValidateChangefeedID(changefeedID); err != nil { - c.IndentedJSON(http.StatusBadRequest, - model.NewHTTPError(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID))) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID)) return } // check if the changefeed exists _, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - return - } - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -522,18 +482,13 @@ func (h *HTTPHandler) MoveTable(c *gin.Context) { ctx := c.Request.Context() changefeedID := c.Param(apiOpVarChangefeedID) if err := model.ValidateChangefeedID(changefeedID); err != nil { - c.IndentedJSON(http.StatusBadRequest, - model.NewHTTPError(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID))) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID)) return } // check if the changefeed exists _, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - return - } - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -543,12 +498,12 @@ func (h *HTTPHandler) MoveTable(c *gin.Context) { }{} err = c.BindJSON(&data) if err != nil { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) + _ = c.Error(cerror.ErrAPIInvalidParam.Wrap(err)) + return } if err := model.ValidateChangefeedID(data.CaptureID); err != nil { - c.IndentedJSON(http.StatusBadRequest, - model.NewHTTPError(cerror.ErrAPIInvalidParam.GenWithStack("invalid capture_id: %s", data.CaptureID))) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid capture_id: %s", data.CaptureID)) return } @@ -603,42 +558,34 @@ func (h *HTTPHandler) GetProcessor(c *gin.Context) { changefeedID := c.Param(apiOpVarChangefeedID) if err := model.ValidateChangefeedID(changefeedID); err != nil { - c.IndentedJSON(http.StatusBadRequest, - model.NewHTTPError(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID))) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID)) return } captureID := c.Param(apiOpVarCaptureID) if err := model.ValidateChangefeedID(captureID); err != nil { - c.IndentedJSON(http.StatusBadRequest, - model.NewHTTPError(cerror.ErrAPIInvalidParam.GenWithStack("invalid capture_id: %s", changefeedID))) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid capture_id: %s", captureID)) return } statuses, err := statusProvider.GetAllTaskStatuses(ctx, changefeedID) if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - } - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } status, exist := statuses[captureID] if !exist { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(cerror.ErrCaptureNotExist.GenWithStackByArgs(captureID))) + _ = c.Error(cerror.ErrCaptureNotExist.GenWithStackByArgs(captureID)) } positions, err := statusProvider.GetTaskPositions(ctx, changefeedID) if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - } - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } position, exist := positions[captureID] if !exist { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(cerror.ErrCaptureNotExist.GenWithStackByArgs(captureID))) + _ = c.Error(cerror.ErrCaptureNotExist.GenWithStackByArgs(captureID)) } processorDetail := &model.ProcessorDetail{CheckPointTs: position.CheckPointTs, ResolvedTs: position.ResolvedTs, Error: position.Error} @@ -647,7 +594,7 @@ func (h *HTTPHandler) GetProcessor(c *gin.Context) { tables = append(tables, tableID) } processorDetail.Tables = tables - c.JSON(http.StatusOK, processorDetail) + c.IndentedJSON(http.StatusOK, processorDetail) } // ListProcessor lists all processors in the TiCDC cluster @@ -669,7 +616,7 @@ func (h *HTTPHandler) ListProcessor(c *gin.Context) { ctx := c.Request.Context() infos, err := statusProvider.GetProcessors(ctx) if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } resps := make([]*model.ProcessorCommonInfo, len(infos)) @@ -699,7 +646,7 @@ func (h *HTTPHandler) ListCapture(c *gin.Context) { ctx := c.Request.Context() captureInfos, err := statusProvider.GetCaptures(ctx) if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -781,14 +728,13 @@ func SetLogLevel(c *gin.Context) { }{} err := c.BindJSON(&data) if err != nil { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid log level: %s", err.Error())) return } err = logutil.SetLogLevel(data.Level) if err != nil { - c.IndentedJSON(http.StatusBadRequest, - model.NewHTTPError(cerror.ErrAPIInvalidParam.GenWithStack("fail to change log level: %s", err))) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("fail to change log level: %s", data.Level)) return } log.Warn("log level changed", zap.String("level", data.Level)) @@ -800,7 +746,7 @@ func (h *HTTPHandler) forwardToOwner(c *gin.Context) { ctx := c.Request.Context() // every request can only forward to owner one time if len(c.GetHeader(forWardFromCapture)) != 0 { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(cerror.ErrRequestForwardErr.FastGenByArgs())) + _ = c.Error(cerror.ErrRequestForwardErr.FastGenByArgs()) return } c.Header(forWardFromCapture, h.capture.Info().ID) @@ -817,13 +763,13 @@ func (h *HTTPHandler) forwardToOwner(c *gin.Context) { return nil }, retry.WithBackoffBaseDelay(300), retry.WithMaxTries(getOwnerRetryMaxTime)) if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } tslConfig, err := config.GetGlobalServerConfig().Security.ToTLSConfigWithVerify() if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -845,7 +791,7 @@ func (h *HTTPHandler) forwardToOwner(c *gin.Context) { cli := httputil.NewClient(tslConfig) resp, err := cli.Do(req) if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -863,7 +809,7 @@ func (h *HTTPHandler) forwardToOwner(c *gin.Context) { defer resp.Body.Close() _, err = bufio.NewReader(resp.Body).WriteTo(c.Writer) if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } } diff --git a/cdc/capture/http_validator.go b/cdc/capture/http_validator.go index 442c5ca3497..e1de06b6711 100644 --- a/cdc/capture/http_validator.go +++ b/cdc/capture/http_validator.go @@ -139,7 +139,7 @@ func verifyCreateChangefeedConfig(ctx context.Context, changefeedConfig model.Ch tz, err := util.GetTimezone(changefeedConfig.TimeZone) if err != nil { - return nil, errors.Annotate(err, "invalid timezone:"+changefeedConfig.TimeZone) + return nil, cerror.ErrAPIInvalidParam.Wrap(errors.Annotatef(err, "invalid timezone:%s", changefeedConfig.TimeZone)) } ctx = util.PutTimezoneInCtx(ctx, tz) if err := sink.Validate(ctx, info.SinkURI, info.Config, info.Opts); err != nil { diff --git a/cdc/http_router.go b/cdc/http_router.go index d6b74cb7aac..8a32b73fa3b 100644 --- a/cdc/http_router.go +++ b/cdc/http_router.go @@ -21,23 +21,29 @@ import ( "time" "github.com/gin-gonic/gin" + "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/capture" + "github.com/pingcap/ticdc/cdc/model" swaggerFiles "github.com/swaggo/files" ginSwagger "github.com/swaggo/gin-swagger" + "go.uber.org/zap" // use for OpenAPI online docs _ "github.com/pingcap/ticdc/docs/api" ) // newRouter create a router for OpenAPI + func newRouter(captureHandler capture.HTTPHandler) *gin.Engine { // discard gin log output gin.DefaultWriter = io.Discard router := gin.New() + router.Use(logMiddleware()) // request will timeout after 10 second router.Use(timeoutMiddleware(time.Second * 10)) + router.Use(errorHandleMiddleware()) // OpenAPI online docs router.GET("/swagger/*any", ginSwagger.WrapHandler(swaggerFiles.Handler)) @@ -96,7 +102,7 @@ func newRouter(captureHandler capture.HTTPHandler) *gin.Engine { } // timeoutMiddleware wraps the request context with a timeout -func timeoutMiddleware(timeout time.Duration) func(c *gin.Context) { +func timeoutMiddleware(timeout time.Duration) gin.HandlerFunc { return func(c *gin.Context) { // wrap the request context with a timeout ctx, cancel := context.WithTimeout(c.Request.Context(), timeout) @@ -119,3 +125,51 @@ func timeoutMiddleware(timeout time.Duration) func(c *gin.Context) { c.Next() } } + +func logMiddleware() gin.HandlerFunc { + return func(c *gin.Context) { + start := time.Now() + path := c.Request.URL.Path + query := c.Request.URL.RawQuery + c.Next() + + cost := time.Since(start) + + err := c.Errors.Last() + var stdErr error + if err != nil { + stdErr = err.Err + } + + log.Info(path, + zap.Int("status", c.Writer.Status()), + zap.String("method", c.Request.Method), + zap.String("path", path), + zap.String("query", query), + zap.String("ip", c.ClientIP()), + zap.String("user-agent", c.Request.UserAgent()), + zap.Error(stdErr), + zap.Duration("cost", cost), + ) + } +} + +func errorHandleMiddleware() gin.HandlerFunc { + return func(c *gin.Context) { + c.Next() + // because we will return immediately after an error occurs in http_handler + // there wil be only one error in c.Errors + lastError := c.Errors.Last() + if lastError != nil { + err := lastError.Err + // put the error into response + if capture.IsHTTPBadRequestError(err) { + c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) + } else { + c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + } + c.Abort() + return + } + } +} diff --git a/cdc/http_router_test.go b/cdc/http_router_test.go index 4821a72b6d2..77e9e184c17 100644 --- a/cdc/http_router_test.go +++ b/cdc/http_router_test.go @@ -23,9 +23,8 @@ import ( "github.com/stretchr/testify/require" ) -func TestPProfRouter(t *testing.T) { +func TestPProfPath(t *testing.T) { t.Parallel() - router := newRouter(capture.NewHTTPHandler(nil)) apis := []*openAPI{ diff --git a/cdc/http_status.go b/cdc/http_status.go index 5aff78126a0..29cf8fd9cfa 100644 --- a/cdc/http_status.go +++ b/cdc/http_status.go @@ -39,6 +39,7 @@ import ( ) func (s *Server) startStatusHTTP() error { + conf := config.GetGlobalServerConfig() router := newRouter(capture.NewHTTPHandler(s.capture)) router.GET("/status", gin.WrapF(s.handleStatus)) @@ -58,7 +59,6 @@ func (s *Server) startStatusHTTP() error { prometheus.DefaultGatherer = registry router.Any("/metrics", gin.WrapH(promhttp.Handler())) - conf := config.GetGlobalServerConfig() err := conf.Security.AddSelfCommonName() if err != nil { log.Error("status server set tls config failed", zap.Error(err)) diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index 9e66b57f5ad..47d5944a180 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -334,7 +334,7 @@ func checkTiDBVariable(ctx context.Context, db *sql.DB, variableName, defaultVal err := db.QueryRowContext(ctx, querySQL).Scan(&name, &value) if err != nil && err != sql.ErrNoRows { errMsg := "fail to query session variable " + variableName - return "", errors.Annotate(cerror.WrapError(cerror.ErrMySQLQueryError, err), errMsg) + return "", cerror.ErrMySQLQueryError.Wrap(err).GenWithStack(errMsg) } // session variable works, use given default value if err == nil { @@ -438,13 +438,12 @@ func parseSinkURI(ctx context.Context, sinkURI *url.URL, opts map[string]string) } tlsCfg, err := credential.ToTLSConfig() if err != nil { - return nil, errors.Annotate(err, "fail to open MySQL connection") + return nil, errors.Trace(err) } name := "cdc_mysql_tls" + params.changefeedID err = dmysql.RegisterTLSConfig(name, tlsCfg) if err != nil { - return nil, errors.Annotate( - cerror.WrapError(cerror.ErrMySQLConnectionError, err), "fail to open MySQL connection") + return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") } params.tls = "?tls=" + name } @@ -513,8 +512,7 @@ var GetDBConnImpl = getDBConn func getDBConn(ctx context.Context, dsnStr string) (*sql.DB, error) { db, err := sql.Open("mysql", dsnStr) if err != nil { - return nil, errors.Annotate( - cerror.WrapError(cerror.ErrMySQLConnectionError, err), "Open database connection failed") + return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") } err = db.PingContext(ctx) if err != nil { @@ -522,8 +520,7 @@ func getDBConn(ctx context.Context, dsnStr string) (*sql.DB, error) { if closeErr := db.Close(); closeErr != nil { log.Warn("close db failed", zap.Error(err)) } - return nil, errors.Annotate( - cerror.WrapError(cerror.ErrMySQLConnectionError, err), "fail to open MySQL connection") + return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") } return db, nil } @@ -1361,12 +1358,12 @@ func newMySQLSyncpointStore(ctx context.Context, id string, sinkURI *url.URL) (S } tlsCfg, err := credential.ToTLSConfig() if err != nil { - return nil, errors.Annotate(err, "fail to open MySQL connection") + return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") } name := "cdc_mysql_tls" + "syncpoint" + id err = dmysql.RegisterTLSConfig(name, tlsCfg) if err != nil { - return nil, errors.Annotate(err, "fail to open MySQL connection") + return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") } tlsParam = "?tls=" + name } @@ -1406,8 +1403,7 @@ func newMySQLSyncpointStore(ctx context.Context, id string, sinkURI *url.URL) (S } testDB, err := sql.Open("mysql", dsn.FormatDSN()) if err != nil { - return nil, errors.Annotate( - cerror.WrapError(cerror.ErrMySQLConnectionError, err), "fail to open MySQL connection when configuring sink") + return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection when configuring sink") } defer testDB.Close() dsnStr, err = configureSinkURI(ctx, dsn, params, testDB) @@ -1416,11 +1412,11 @@ func newMySQLSyncpointStore(ctx context.Context, id string, sinkURI *url.URL) (S } syncDB, err = sql.Open("mysql", dsnStr) if err != nil { - return nil, errors.Annotate(err, "Open database connection failed") + return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") } err = syncDB.PingContext(ctx) if err != nil { - return nil, errors.Annotate(err, "fail to open MySQL connection") + return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") } log.Info("Start mysql syncpoint sink") diff --git a/cdc/sink/simple_mysql_tester.go b/cdc/sink/simple_mysql_tester.go index 3b3d7edc885..d68f538802c 100644 --- a/cdc/sink/simple_mysql_tester.go +++ b/cdc/sink/simple_mysql_tester.go @@ -85,7 +85,7 @@ func newSimpleMySQLSink(ctx context.Context, sinkURI *url.URL, config *config.Re db, err = sql.Open("mysql", dsnStr) if err != nil { return nil, errors.Annotate( - cerror.WrapError(cerror.ErrMySQLConnectionError, err), "Open database connection failed") + cerror.WrapError(cerror.ErrMySQLConnectionError, err), "fail to open MySQL connection") } err = db.PingContext(ctx) if err != nil { diff --git a/cdc/sink/sink_test.go b/cdc/sink/sink_test.go index 557239a9861..808b7324577 100644 --- a/cdc/sink/sink_test.go +++ b/cdc/sink/sink_test.go @@ -35,7 +35,7 @@ func TestValidateSink(t *testing.T) { sinkURI := "mysql://root:111@127.0.0.1:3306/" err := Validate(ctx, sinkURI, replicateConfig, opts) require.NotNil(t, err) - require.Regexp(t, "fail to open MySQL connection.*ErrMySQLConnectionError.*", err) + require.Contains(t, err.Error(), "fail to open MySQL connection") // test sink uri right sinkURI = "blackhole://" diff --git a/pkg/config/config_test.go b/pkg/config/config_test.go index 05f523b33b7..09430135d3a 100644 --- a/pkg/config/config_test.go +++ b/pkg/config/config_test.go @@ -69,6 +69,7 @@ func TestReplicaConfigOutDated(t *testing.T) { func TestServerConfigMarshal(t *testing.T) { t.Parallel() + rawConfig := `{"addr":"192.155.22.33:8887","advertise-addr":"","log-file":"","log-level":"info","log":{"file":{"max-size":300,"max-days":0,"max-backups":0}},"data-dir":"","gc-ttl":86400,"tz":"System","capture-session-ttl":10,"owner-flush-interval":200000000,"processor-flush-interval":100000000,"sorter":{"num-concurrent-worker":4,"chunk-size-limit":999,"max-memory-percentage":30,"max-memory-consumption":17179869184,"num-workerpool-goroutine":16,"sort-dir":"/tmp/sorter"},"security":{"ca-path":"","cert-path":"","key-path":"","cert-allowed-cn":null},"per-table-memory-quota":10485760,"kv-client":{"worker-concurrent":8,"worker-pool-size":0,"region-scan-limit":40}}` conf := GetDefaultServerConfig() From 9a8eaf07209dc1668468e6657aaae6eecf7a1e4a Mon Sep 17 00:00:00 2001 From: Neil Shen Date: Fri, 5 Nov 2021 20:55:04 +0800 Subject: [PATCH 4/5] leveldb: add message, config and metrics (#3272) --- cdc/metrics.go | 4 + cdc/sorter/leveldb/message/task.go | 56 +++++++ cdc/sorter/leveldb/message/task_test.go | 37 +++++ cdc/sorter/leveldb/metrics.go | 50 ++++++ cdc/sorter/metrics.go | 69 ++++++++ cdc/sorter/unified/backend_pool.go | 7 +- cdc/sorter/unified/merger.go | 5 +- cdc/sorter/unified/metrics.go | 40 ----- errors.toml | 4 +- go.mod | 2 +- go.sum | 5 +- pkg/actor/message/message.go | 23 +++ pkg/actor/message/message_test.go | 8 + pkg/cmd/server/server_test.go | 60 +++++++ pkg/config/config.go | 20 +++ pkg/config/config_data_test.go | 199 ++++++++++++++++++++++++ pkg/config/config_test.go | 35 ++++- pkg/config/sorter.go | 81 +++++++++- pkg/errors/errors.go | 6 +- 19 files changed, 647 insertions(+), 64 deletions(-) create mode 100644 cdc/sorter/leveldb/message/task.go create mode 100644 cdc/sorter/leveldb/message/task_test.go create mode 100644 cdc/sorter/leveldb/metrics.go create mode 100644 cdc/sorter/metrics.go create mode 100644 pkg/config/config_data_test.go diff --git a/cdc/metrics.go b/cdc/metrics.go index ae7cfac34e7..220de00f247 100644 --- a/cdc/metrics.go +++ b/cdc/metrics.go @@ -21,6 +21,8 @@ import ( tablepipeline "github.com/pingcap/ticdc/cdc/processor/pipeline" "github.com/pingcap/ticdc/cdc/puller" "github.com/pingcap/ticdc/cdc/sink" + "github.com/pingcap/ticdc/cdc/sorter" + "github.com/pingcap/ticdc/cdc/sorter/leveldb" "github.com/pingcap/ticdc/cdc/sorter/memory" "github.com/pingcap/ticdc/cdc/sorter/unified" "github.com/pingcap/ticdc/pkg/actor" @@ -45,6 +47,8 @@ func init() { initServerMetrics(registry) actor.InitMetrics(registry) // Sorter metrics + sorter.InitMetrics(registry) memory.InitMetrics(registry) unified.InitMetrics(registry) + leveldb.InitMetrics(registry) } diff --git a/cdc/sorter/leveldb/message/task.go b/cdc/sorter/leveldb/message/task.go new file mode 100644 index 00000000000..24795e94067 --- /dev/null +++ b/cdc/sorter/leveldb/message/task.go @@ -0,0 +1,56 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package message + +import ( + "fmt" + + "github.com/pingcap/ticdc/cdc/sorter/encoding" + Iterator "github.com/syndtr/goleveldb/leveldb/iterator" + lutil "github.com/syndtr/goleveldb/leveldb/util" + "golang.org/x/sync/semaphore" +) + +// Task is a leveldb actor task. It carries write and read request. +type Task struct { + UID uint32 + TableID uint64 + + // encoded key -> serde.marshal(event) + // If a value is empty, it deletes the key/value entry in leveldb. + Events map[Key][]byte + // Must be buffered channel to avoid blocking. + IterCh chan LimitedIterator `json:"-"` // Make Task JSON printable. + Irange *lutil.Range + // Set NeedIter whenever caller wants to read something from an iterator. + NeedIter bool +} + +// Key is the key that is written to leveldb. +type Key string + +// String returns a pretty printed string. +func (k Key) String() string { + uid, tableID, startTs, CRTs := encoding.DecodeKey([]byte(k)) + return fmt.Sprintf( + "uid: %d, tableID: %d, startTs: %d, CRTs: %d", + uid, tableID, startTs, CRTs) +} + +// LimitedIterator is a wrapper of leveldb iterator that has a sema to limit +// the total number of open iterators. +type LimitedIterator struct { + Iterator.Iterator + Sema *semaphore.Weighted +} diff --git a/cdc/sorter/leveldb/message/task_test.go b/cdc/sorter/leveldb/message/task_test.go new file mode 100644 index 00000000000..8b0de95283c --- /dev/null +++ b/cdc/sorter/leveldb/message/task_test.go @@ -0,0 +1,37 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package message + +import ( + "testing" + + "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/cdc/sorter/encoding" + "github.com/stretchr/testify/require" +) + +func TestPrint(t *testing.T) { + t.Parallel() + event := model.NewPolymorphicEvent(&model.RawKVEntry{ + OpType: model.OpTypeDelete, + Key: []byte{1}, + StartTs: 3, + CRTs: 4, + }) + + require.Equal(t, "uid: 1, tableID: 2, startTs: 3, CRTs: 4", + Key(encoding.EncodeKey(1, 2, event)).String()) + require.Equal(t, "uid: 1, tableID: 2, startTs: 0, CRTs: 3", + Key(encoding.EncodeTsKey(1, 2, 3)).String()) +} diff --git a/cdc/sorter/leveldb/metrics.go b/cdc/sorter/leveldb/metrics.go new file mode 100644 index 00000000000..65dd6cb0001 --- /dev/null +++ b/cdc/sorter/leveldb/metrics.go @@ -0,0 +1,50 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package leveldb + +import ( + "github.com/prometheus/client_golang/prometheus" +) + +var ( + sorterWriteBytesHistogram = prometheus.NewHistogramVec(prometheus.HistogramOpts{ + Namespace: "ticdc", + Subsystem: "sorter", + Name: "leveldb_write_bytes", + Help: "Bucketed histogram of sorter write batch bytes", + Buckets: prometheus.ExponentialBuckets(16, 2.0, 20), + }, []string{"capture", "id"}) + + sorterWriteDurationHistogram = prometheus.NewHistogramVec(prometheus.HistogramOpts{ + Namespace: "ticdc", + Subsystem: "sorter", + Name: "leveldb_write_duration_seconds", + Help: "Bucketed histogram of sorter write duration", + Buckets: prometheus.ExponentialBuckets(0.004, 2.0, 20), + }, []string{"capture", "id"}) + + sorterCleanupKVCounter = prometheus.NewCounterVec(prometheus.CounterOpts{ + Namespace: "ticdc", + Subsystem: "sorter", + Name: "leveldb_cleanup_kv_total", + Help: "The total number of cleaned up kv entries", + }, []string{"capture", "id"}) +) + +// InitMetrics registers all metrics in this file +func InitMetrics(registry *prometheus.Registry) { + registry.MustRegister(sorterWriteDurationHistogram) + registry.MustRegister(sorterWriteBytesHistogram) + registry.MustRegister(sorterCleanupKVCounter) +} diff --git a/cdc/sorter/metrics.go b/cdc/sorter/metrics.go new file mode 100644 index 00000000000..f4e30109101 --- /dev/null +++ b/cdc/sorter/metrics.go @@ -0,0 +1,69 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package sorter + +import ( + "github.com/prometheus/client_golang/prometheus" +) + +var ( + // SorterEventCount is the metric that counts events output by the sorter. + SorterEventCount = prometheus.NewCounterVec(prometheus.CounterOpts{ + Namespace: "ticdc", + Subsystem: "sorter", + Name: "event_count", + Help: "The number of events output by the sorter", + }, []string{"capture", "changefeed", "type"}) + + // SorterResolvedTsGauge is the metric that records sorter resolved ts. + SorterResolvedTsGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sorter", + Name: "resolved_ts_gauge", + Help: "the resolved ts of the sorter", + }, []string{"capture", "changefeed"}) + + // SorterInMemoryDataSizeGauge is the metric that records sorter memory usage. + SorterInMemoryDataSizeGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sorter", + Name: "in_memory_data_size_gauge", + Help: "The amount of pending data stored in-memory by the sorter", + }, []string{"capture", "id"}) + + // SorterOnDiskDataSizeGauge is the metric that records sorter disk usage. + SorterOnDiskDataSizeGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sorter", + Name: "on_disk_data_size_gauge", + Help: "The amount of pending data stored on-disk by the sorter", + }, []string{"capture", "id"}) + + // SorterOpenFileCountGauge is the metric that records sorter open files. + SorterOpenFileCountGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sorter", + Name: "open_file_count_gauge", + Help: "The number of open file descriptors held by the sorter", + }, []string{"capture", "id"}) +) + +// InitMetrics registers all metrics in this file +func InitMetrics(registry *prometheus.Registry) { + registry.MustRegister(SorterEventCount) + registry.MustRegister(SorterResolvedTsGauge) + registry.MustRegister(SorterInMemoryDataSizeGauge) + registry.MustRegister(SorterOnDiskDataSizeGauge) + registry.MustRegister(SorterOpenFileCountGauge) +} diff --git a/cdc/sorter/unified/backend_pool.go b/cdc/sorter/unified/backend_pool.go index 3f5add5f70b..689d423e742 100644 --- a/cdc/sorter/unified/backend_pool.go +++ b/cdc/sorter/unified/backend_pool.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/pingcap/ticdc/cdc/sorter" sorterencoding "github.com/pingcap/ticdc/cdc/sorter/encoding" "github.com/pingcap/ticdc/pkg/config" cerrors "github.com/pingcap/ticdc/pkg/errors" @@ -93,9 +94,9 @@ func newBackEndPool(dir string, captureAddr string) (*backEndPool, error) { ticker := time.NewTicker(backgroundJobInterval) defer ticker.Stop() - metricSorterInMemoryDataSizeGauge := sorterInMemoryDataSizeGauge.WithLabelValues(captureAddr) - metricSorterOnDiskDataSizeGauge := sorterOnDiskDataSizeGauge.WithLabelValues(captureAddr) - metricSorterOpenFileCountGauge := sorterOpenFileCountGauge.WithLabelValues(captureAddr) + metricSorterInMemoryDataSizeGauge := sorter.SorterInMemoryDataSizeGauge.WithLabelValues(captureAddr, "0") + metricSorterOnDiskDataSizeGauge := sorter.SorterOnDiskDataSizeGauge.WithLabelValues(captureAddr, "0") + metricSorterOpenFileCountGauge := sorter.SorterOpenFileCountGauge.WithLabelValues(captureAddr, "0") for { select { diff --git a/cdc/sorter/unified/merger.go b/cdc/sorter/unified/merger.go index 97ff7baa9ac..e331f1c125e 100644 --- a/cdc/sorter/unified/merger.go +++ b/cdc/sorter/unified/merger.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/cdc/sorter" cerrors "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/notify" "github.com/pingcap/ticdc/pkg/util" @@ -39,11 +40,11 @@ func runMerger(ctx context.Context, numSorters int, in <-chan *flushTask, out ch captureAddr := util.CaptureAddrFromCtx(ctx) changefeedID := util.ChangefeedIDFromCtx(ctx) - metricSorterEventCount := sorterEventCount.MustCurryWith(map[string]string{ + metricSorterEventCount := sorter.SorterEventCount.MustCurryWith(map[string]string{ "capture": captureAddr, "changefeed": changefeedID, }) - metricSorterResolvedTsGauge := sorterResolvedTsGauge.WithLabelValues(captureAddr, changefeedID) + metricSorterResolvedTsGauge := sorter.SorterResolvedTsGauge.WithLabelValues(captureAddr, changefeedID) metricSorterMergerStartTsGauge := sorterMergerStartTsGauge.WithLabelValues(captureAddr, changefeedID) metricSorterMergeCountHistogram := sorterMergeCountHistogram.WithLabelValues(captureAddr, changefeedID) diff --git a/cdc/sorter/unified/metrics.go b/cdc/sorter/unified/metrics.go index d510ccf78a1..d4f9a23e2cf 100644 --- a/cdc/sorter/unified/metrics.go +++ b/cdc/sorter/unified/metrics.go @@ -25,20 +25,6 @@ var ( Help: "the number of events consumed by the sorter", }, []string{"capture", "changefeed", "type"}) - sorterEventCount = prometheus.NewCounterVec(prometheus.CounterOpts{ - Namespace: "ticdc", - Subsystem: "sorter", - Name: "event_count", - Help: "the number of events output by the sorter", - }, []string{"capture", "changefeed", "type"}) - - sorterResolvedTsGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "sorter", - Name: "resolved_ts_gauge", - Help: "the resolved ts of the sorter", - }, []string{"capture", "changefeed"}) - sorterMergerStartTsGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ Namespace: "ticdc", Subsystem: "sorter", @@ -46,27 +32,6 @@ var ( Help: "the start TS of each merge in the sorter", }, []string{"capture", "changefeed"}) - sorterInMemoryDataSizeGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "sorter", - Name: "in_memory_data_size_gauge", - Help: "the amount of pending data stored in-memory by the sorter", - }, []string{"capture"}) - - sorterOnDiskDataSizeGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "sorter", - Name: "on_disk_data_size_gauge", - Help: "the amount of pending data stored on-disk by the sorter", - }, []string{"capture"}) - - sorterOpenFileCountGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "sorter", - Name: "open_file_count_gauge", - Help: "the number of open file descriptors held by the sorter", - }, []string{"capture"}) - sorterFlushCountHistogram = prometheus.NewHistogramVec(prometheus.HistogramOpts{ Namespace: "ticdc", Subsystem: "sorter", @@ -87,12 +52,7 @@ var ( // InitMetrics registers all metrics in this file func InitMetrics(registry *prometheus.Registry) { registry.MustRegister(sorterConsumeCount) - registry.MustRegister(sorterEventCount) - registry.MustRegister(sorterResolvedTsGauge) registry.MustRegister(sorterMergerStartTsGauge) - registry.MustRegister(sorterInMemoryDataSizeGauge) - registry.MustRegister(sorterOnDiskDataSizeGauge) - registry.MustRegister(sorterOpenFileCountGauge) registry.MustRegister(sorterFlushCountHistogram) registry.MustRegister(sorterMergeCountHistogram) } diff --git a/errors.toml b/errors.toml index 57860d8a95a..72d977fa499 100755 --- a/errors.toml +++ b/errors.toml @@ -331,9 +331,9 @@ error = ''' get tikv grpc context failed ''' -["CDC:ErrIllegalUnifiedSorterParameter"] +["CDC:ErrIllegalSorterParameter"] error = ''' -illegal parameter for unified sorter: %s +illegal parameter for sorter: %s ''' ["CDC:ErrIndexKeyTableNotFound"] diff --git a/go.mod b/go.mod index eeacc6091c8..9e730fe431c 100644 --- a/go.mod +++ b/go.mod @@ -70,7 +70,7 @@ require ( github.com/swaggo/files v0.0.0-20190704085106-630677cd5c14 github.com/swaggo/gin-swagger v1.2.0 github.com/swaggo/swag v1.6.6-0.20200529100950-7c765ddd0476 - github.com/syndtr/goleveldb v1.0.1-0.20190318030020-c3a204f8e965 + github.com/syndtr/goleveldb v1.0.1-0.20210305035536-64b5b1c73954 github.com/tidwall/gjson v1.9.1 github.com/tidwall/sjson v1.2.2 github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211011083157-49c8dd23f1f0 diff --git a/go.sum b/go.sum index b6b572356d4..b9ea6b4f78b 100644 --- a/go.sum +++ b/go.sum @@ -920,8 +920,9 @@ github.com/swaggo/swag v1.5.1/go.mod h1:1Bl9F/ZBpVWh22nY0zmYyASPO1lI/zIwRDrpZU+t github.com/swaggo/swag v1.6.3/go.mod h1:wcc83tB4Mb2aNiL/HP4MFeQdpHUrca+Rp/DRNgWAUio= github.com/swaggo/swag v1.6.6-0.20200529100950-7c765ddd0476 h1:UjnSXdNPIG+5FJ6xLQODEdk7gSnJlMldu3sPAxxCO+4= github.com/swaggo/swag v1.6.6-0.20200529100950-7c765ddd0476/go.mod h1:xDhTyuFIujYiN3DKWC/H/83xcfHp+UE/IzWWampG7Zc= -github.com/syndtr/goleveldb v1.0.1-0.20190318030020-c3a204f8e965 h1:1oFLiOyVl+W7bnBzGhf7BbIv9loSFQcieWWYIjLqcAw= github.com/syndtr/goleveldb v1.0.1-0.20190318030020-c3a204f8e965/go.mod h1:9OrXJhf154huy1nPWmuSrkgjPUtUNhA+Zmy+6AESzuA= +github.com/syndtr/goleveldb v1.0.1-0.20210305035536-64b5b1c73954 h1:xQdMZ1WLrgkkvOZ/LDQxjVxMLdby7osSh4ZEVa5sIjs= +github.com/syndtr/goleveldb v1.0.1-0.20210305035536-64b5b1c73954/go.mod h1:u2MKkTVTVJWe5D1rCvame8WqhBd88EuIwODJZ1VHCPM= github.com/thoas/go-funk v0.7.0/go.mod h1:+IWnUfUmFO1+WVYQWQtIJHeRRdaIyyYglZN7xzUPe4Q= github.com/thoas/go-funk v0.8.0/go.mod h1:+IWnUfUmFO1+WVYQWQtIJHeRRdaIyyYglZN7xzUPe4Q= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= @@ -1172,6 +1173,7 @@ golang.org/x/net v0.0.0-20200520182314-0ba52f642ac2/go.mod h1:qpuaurCH72eLCgpAm/ golang.org/x/net v0.0.0-20200602114024-627f9648deb9/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20200707034311-ab3426394381/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20200813134508-3edf25e44fcc/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20200904194848-62affa334b73/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= @@ -1266,6 +1268,7 @@ golang.org/x/sys v0.0.0-20200523222454-059865788121/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200803210538-64077c9b5642/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200826173525-f9321e4c35a6/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200814200057-3d37ad5750ed/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200905004654-be1d3432aa8f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= diff --git a/pkg/actor/message/message.go b/pkg/actor/message/message.go index 3e44a4f4604..ff88692305e 100644 --- a/pkg/actor/message/message.go +++ b/pkg/actor/message/message.go @@ -15,6 +15,7 @@ package message import ( "github.com/pingcap/ticdc/cdc/model" + sorter "github.com/pingcap/ticdc/cdc/sorter/leveldb/message" ) // Type is the type of Message @@ -24,7 +25,9 @@ type Type int const ( TypeUnknown Type = iota TypeTick + TypeStop TypeBarrier + TypeSorterTask // Add a new type when adding a new message. ) @@ -34,6 +37,11 @@ type Message struct { Tp Type // BarrierTs BarrierTs model.Ts + // Leveldb sorter task + // TODO: find a way to hide it behind an interface while saving + // memory allocation. + // See https://cs.opensource.google/go/go/+/refs/tags/go1.17.2:src/runtime/iface.go;l=325 + SorterTask sorter.Task } // TickMessage creates the message of Tick @@ -43,6 +51,13 @@ func TickMessage() Message { } } +// StopMessage creates the message of Stop +func StopMessage() Message { + return Message{ + Tp: TypeStop, + } +} + // BarrierMessage creates the message of Command func BarrierMessage(barrierTs model.Ts) Message { return Message{ @@ -50,3 +65,11 @@ func BarrierMessage(barrierTs model.Ts) Message { BarrierTs: barrierTs, } } + +// SorterMessage creates the message of sorter +func SorterMessage(task sorter.Task) Message { + return Message{ + Tp: TypeSorterTask, + SorterTask: task, + } +} diff --git a/pkg/actor/message/message_test.go b/pkg/actor/message/message_test.go index 4551ee9dfa1..18f1e575d88 100644 --- a/pkg/actor/message/message_test.go +++ b/pkg/actor/message/message_test.go @@ -17,6 +17,7 @@ import ( "encoding/json" "testing" + sorter "github.com/pingcap/ticdc/cdc/sorter/leveldb/message" "github.com/pingcap/ticdc/pkg/leakutil" "github.com/stretchr/testify/require" ) @@ -41,3 +42,10 @@ func TestBarrierMessage(t *testing.T) { msg := BarrierMessage(1) require.Equal(t, TypeBarrier, msg.Tp) } + +func TestSorterMessage(t *testing.T) { + task := sorter.Task{UID: 1, TableID: 2} + msg := SorterMessage(task) + require.Equal(t, TypeSorterTask, msg.Tp) + require.Equal(t, task, msg.SorterTask) +} diff --git a/pkg/cmd/server/server_test.go b/pkg/cmd/server/server_test.go index 08cac96c4cf..4286e4ee26b 100644 --- a/pkg/cmd/server/server_test.go +++ b/pkg/cmd/server/server_test.go @@ -15,6 +15,7 @@ package server import ( "fmt" + "math" "os" "path/filepath" "testing" @@ -174,6 +175,21 @@ func (s *serverSuite) TestParseCfg(c *check.C) { MaxMemoryConsumption: 60000, NumWorkerPoolGoroutine: 90, SortDir: config.DefaultSortDir, + EnableLevelDB: false, + LevelDB: config.LevelDBConfig{ + LevelDBCount: 16, + LevelDBConcurrency: 256, + MaxOpenFiles: 10000, + BlockSize: 65536, + BlockCacheSize: 0, + WriterBufferSize: 8388608, + Compression: "snappy", + TargetFileSizeBase: 8388608, + CompactionL0Trigger: 160, + WriteL0SlowdownTrigger: math.MaxInt32, + WriteL0PauseTrigger: math.MaxInt32, + CleanupSpeedLimit: 10000, + }, }, Security: &config.SecurityConfig{ CertPath: "bb", @@ -221,6 +237,20 @@ max-memory-percentage = 3 num-concurrent-worker = 4 num-workerpool-goroutine = 5 sort-dir = "/tmp/just_a_test" +enable-leveldb-sorter = false +[sorter.leveldb] +leveldb-count = 5 +leveldb-concurrency = 6 +max-open-files = 7 +block-size = 32768 # 32 KB +block-cache-size = 8 +writer-buffer-size = 9 +compression = "none" +target-file-size-base = 10 +compaction-l0-trigger = 11 +write-l0-slowdown-trigger = 12 +write-l0-pause-trigger = 13 +cleanup-speed-limit = 14 `, dataDir) err := os.WriteFile(configPath, []byte(configContent), 0o644) c.Assert(err, check.IsNil) @@ -260,6 +290,21 @@ sort-dir = "/tmp/just_a_test" MaxMemoryConsumption: 2000000, NumWorkerPoolGoroutine: 5, SortDir: config.DefaultSortDir, + EnableLevelDB: false, + LevelDB: config.LevelDBConfig{ + LevelDBCount: 5, + LevelDBConcurrency: 6, + MaxOpenFiles: 7, + BlockSize: 32768, + BlockCacheSize: 8, + WriterBufferSize: 9, + Compression: "none", + TargetFileSizeBase: 10, + CompactionL0Trigger: 11, + WriteL0SlowdownTrigger: 12, + WriteL0PauseTrigger: 13, + CleanupSpeedLimit: 14, + }, }, Security: &config.SecurityConfig{}, PerTableMemoryQuota: 10 * 1024 * 1024, // 10M @@ -363,6 +408,21 @@ cert-allowed-cn = ["dd","ee"] MaxMemoryConsumption: 60000000, NumWorkerPoolGoroutine: 5, SortDir: config.DefaultSortDir, + EnableLevelDB: false, + LevelDB: config.LevelDBConfig{ + LevelDBCount: 16, + LevelDBConcurrency: 256, + MaxOpenFiles: 10000, + BlockSize: 65536, + BlockCacheSize: 0, + WriterBufferSize: 8388608, + Compression: "snappy", + TargetFileSizeBase: 8388608, + CompactionL0Trigger: 160, + WriteL0SlowdownTrigger: math.MaxInt32, + WriteL0PauseTrigger: math.MaxInt32, + CleanupSpeedLimit: 10000, + }, }, Security: &config.SecurityConfig{ CertPath: "bb", diff --git a/pkg/config/config.go b/pkg/config/config.go index c1c41b61481..9be1f9b6ad7 100644 --- a/pkg/config/config.go +++ b/pkg/config/config.go @@ -16,6 +16,7 @@ package config import ( "encoding/json" "fmt" + "math" "net" "strings" "sync/atomic" @@ -194,6 +195,25 @@ var defaultServerConfig = &ServerConfig{ MaxMemoryConsumption: 16 * 1024 * 1024 * 1024, // 16GB NumWorkerPoolGoroutine: 16, SortDir: DefaultSortDir, + + // Default leveldb sorter config + EnableLevelDB: false, + LevelDB: LevelDBConfig{ + LevelDBCount: 16, + // Following configs are optimized for write throughput. + // Users should not change them. + LevelDBConcurrency: 256, + MaxOpenFiles: 10000, + BlockSize: 65536, + BlockCacheSize: 0, + WriterBufferSize: 8388608, + Compression: "snappy", + TargetFileSizeBase: 8388608, + CompactionL0Trigger: 160, + WriteL0SlowdownTrigger: math.MaxInt32, + WriteL0PauseTrigger: math.MaxInt32, + CleanupSpeedLimit: 10000, + }, }, Security: &SecurityConfig{}, PerTableMemoryQuota: 10 * 1024 * 1024, // 10MB diff --git a/pkg/config/config_data_test.go b/pkg/config/config_data_test.go new file mode 100644 index 00000000000..e0382a47ccc --- /dev/null +++ b/pkg/config/config_data_test.go @@ -0,0 +1,199 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package config + +const ( + testCfgTestReplicaConfigOutDated = `{ + "case-sensitive": false, + "enable-old-value": true, + "force-replicate": true, + "check-gc-safe-point": true, + "filter": { + "rules": [ + "1.1" + ], + "ignore-txn-start-ts": null, + "ddl-allow-list": null + }, + "mounter": { + "worker-num": 3 + }, + "sink": { + "dispatch-rules": [ + { + "db-name": "a", + "tbl-name": "b", + "rule": "r1" + }, + { + "db-name": "a", + "tbl-name": "c", + "rule": "r2" + }, + { + "db-name": "a", + "tbl-name": "d", + "rule": "r2" + } + ], + "protocol": "default" + }, + "cyclic-replication": { + "enable": false, + "replica-id": 0, + "filter-replica-ids": null, + "id-buckets": 0, + "sync-ddl": false + }, + "scheduler": { + "type": "table-number", + "polling-time": -1 + }, + "consistent": { + "level": "none", + "max-log-size": 64, + "flush-interval": 1000, + "storage": "" + } +}` + + testCfgTestServerConfigMarshal = `{ + "addr": "192.155.22.33:8887", + "advertise-addr": "", + "log-file": "", + "log-level": "info", + "log": { + "file": { + "max-size": 300, + "max-days": 0, + "max-backups": 0 + } + }, + "data-dir": "", + "gc-ttl": 86400, + "tz": "System", + "capture-session-ttl": 10, + "owner-flush-interval": 200000000, + "processor-flush-interval": 100000000, + "sorter": { + "num-concurrent-worker": 4, + "chunk-size-limit": 999, + "max-memory-percentage": 30, + "max-memory-consumption": 17179869184, + "num-workerpool-goroutine": 16, + "sort-dir": "/tmp/sorter", + "enable-leveldb-sorter": false, + "leveldb": { + "leveldb-count": 16, + "leveldb-concurrency": 256, + "max-open-files": 10000, + "block-size": 65536, + "block-cache-size": 0, + "writer-buffer-size": 8388608, + "compression": "snappy", + "target-file-size-base": 8388608, + "compaction-l0-trigger": 160, + "write-l0-slowdown-trigger": 2147483647, + "write-l0-pause-trigger": 2147483647, + "cleanup-speed-limit": 10000 + } + }, + "security": { + "ca-path": "", + "cert-path": "", + "key-path": "", + "cert-allowed-cn": null + }, + "per-table-memory-quota": 10485760, + "kv-client": { + "worker-concurrent": 8, + "worker-pool-size": 0, + "region-scan-limit": 40 + } +}` + + testCfgTestReplicaConfigMarshal1 = `{ + "case-sensitive": false, + "enable-old-value": true, + "force-replicate": true, + "check-gc-safe-point": true, + "filter": { + "rules": [ + "1.1" + ], + "ignore-txn-start-ts": null + }, + "mounter": { + "worker-num": 3 + }, + "sink": { + "dispatchers": null, + "protocol": "default" + }, + "cyclic-replication": { + "enable": false, + "replica-id": 0, + "filter-replica-ids": null, + "id-buckets": 0, + "sync-ddl": false + }, + "scheduler": { + "type": "table-number", + "polling-time": -1 + }, + "consistent": { + "level": "none", + "max-log-size": 64, + "flush-interval": 1000, + "storage": "" + } +}` + + testCfgTestReplicaConfigMarshal2 = `{ + "case-sensitive": false, + "enable-old-value": true, + "force-replicate": true, + "check-gc-safe-point": true, + "filter": { + "rules": [ + "1.1" + ], + "ignore-txn-start-ts": null + }, + "mounter": { + "worker-num": 3 + }, + "sink": { + "dispatchers": null, + "protocol": "default" + }, + "cyclic-replication": { + "enable": false, + "replica-id": 0, + "filter-replica-ids": null, + "id-buckets": 0, + "sync-ddl": false + }, + "scheduler": { + "type": "table-number", + "polling-time": -1 + }, + "consistent": { + "level": "none", + "max-log-size": 64, + "flush-interval": 1000, + "storage": "" + } +}` +) diff --git a/pkg/config/config_test.go b/pkg/config/config_test.go index 09430135d3a..e8193db6283 100644 --- a/pkg/config/config_test.go +++ b/pkg/config/config_test.go @@ -14,11 +14,20 @@ package config import ( + "bytes" + "encoding/json" "testing" "github.com/stretchr/testify/require" ) +func mustIdentJSON(t *testing.T, j string) string { + var buf bytes.Buffer + err := json.Indent(&buf, []byte(j), "", " ") + require.Nil(t, err) + return buf.String() +} + func TestReplicaConfigMarshal(t *testing.T) { t.Parallel() conf := GetDefaultReplicaConfig() @@ -28,9 +37,9 @@ func TestReplicaConfigMarshal(t *testing.T) { conf.Mounter.WorkerNum = 3 b, err := conf.Marshal() require.Nil(t, err) - require.Equal(t, `{"case-sensitive":false,"enable-old-value":true,"force-replicate":true,"check-gc-safe-point":true,"filter":{"rules":["1.1"],"ignore-txn-start-ts":null},"mounter":{"worker-num":3},"sink":{"dispatchers":null,"protocol":"default"},"cyclic-replication":{"enable":false,"replica-id":0,"filter-replica-ids":null,"id-buckets":0,"sync-ddl":false},"scheduler":{"type":"table-number","polling-time":-1},"consistent":{"level":"none","max-log-size":64,"flush-interval":1000,"storage":""}}`, b) + require.Equal(t, testCfgTestReplicaConfigMarshal1, mustIdentJSON(t, b)) conf2 := new(ReplicaConfig) - err = conf2.Unmarshal([]byte(`{"case-sensitive":false,"enable-old-value":true,"force-replicate":true,"check-gc-safe-point":true,"filter":{"rules":["1.1"],"ignore-txn-start-ts":null},"mounter":{"worker-num":3},"sink":{"dispatchers":null,"protocol":"default"},"cyclic-replication":{"enable":false,"replica-id":0,"filter-replica-ids":null,"id-buckets":0,"sync-ddl":false},"scheduler":{"type":"table-number","polling-time":-1},"consistent":{"level":"none","max-log-size":64,"flush-interval":1000,"storage":""}}`)) + err = conf2.Unmarshal([]byte(testCfgTestReplicaConfigMarshal2)) require.Nil(t, err) require.Equal(t, conf, conf2) } @@ -51,7 +60,7 @@ func TestReplicaConfigClone(t *testing.T) { func TestReplicaConfigOutDated(t *testing.T) { t.Parallel() conf2 := new(ReplicaConfig) - err := conf2.Unmarshal([]byte(`{"case-sensitive":false,"enable-old-value":true,"force-replicate":true,"check-gc-safe-point":true,"filter":{"rules":["1.1"],"ignore-txn-start-ts":null,"ddl-allow-list":null},"mounter":{"worker-num":3},"sink":{"dispatch-rules":[{"db-name":"a","tbl-name":"b","rule":"r1"},{"db-name":"a","tbl-name":"c","rule":"r2"},{"db-name":"a","tbl-name":"d","rule":"r2"}],"protocol":"default"},"cyclic-replication":{"enable":false,"replica-id":0,"filter-replica-ids":null,"id-buckets":0,"sync-ddl":false},"scheduler":{"type":"table-number","polling-time":-1},"consistent":{"level":"none","max-log-size":64,"flush-interval":1000,"storage":""}}`)) + err := conf2.Unmarshal([]byte(testCfgTestReplicaConfigOutDated)) require.Nil(t, err) conf := GetDefaultReplicaConfig() @@ -69,8 +78,7 @@ func TestReplicaConfigOutDated(t *testing.T) { func TestServerConfigMarshal(t *testing.T) { t.Parallel() - - rawConfig := `{"addr":"192.155.22.33:8887","advertise-addr":"","log-file":"","log-level":"info","log":{"file":{"max-size":300,"max-days":0,"max-backups":0}},"data-dir":"","gc-ttl":86400,"tz":"System","capture-session-ttl":10,"owner-flush-interval":200000000,"processor-flush-interval":100000000,"sorter":{"num-concurrent-worker":4,"chunk-size-limit":999,"max-memory-percentage":30,"max-memory-consumption":17179869184,"num-workerpool-goroutine":16,"sort-dir":"/tmp/sorter"},"security":{"ca-path":"","cert-path":"","key-path":"","cert-allowed-cn":null},"per-table-memory-quota":10485760,"kv-client":{"worker-concurrent":8,"worker-pool-size":0,"region-scan-limit":40}}` + rawConfig := testCfgTestServerConfigMarshal conf := GetDefaultServerConfig() conf.Addr = "192.155.22.33:8887" @@ -78,7 +86,7 @@ func TestServerConfigMarshal(t *testing.T) { b, err := conf.Marshal() require.Nil(t, err) - require.Equal(t, rawConfig, b) + require.Equal(t, rawConfig, mustIdentJSON(t, b)) conf2 := new(ServerConfig) err = conf2.Unmarshal([]byte(rawConfig)) require.Nil(t, err) @@ -124,3 +132,18 @@ func TestServerConfigValidateAndAdjust(t *testing.T) { require.Nil(t, conf.ValidateAndAdjust()) require.EqualValues(t, GetDefaultServerConfig().PerTableMemoryQuota, conf.PerTableMemoryQuota) } + +func TestSorterConfigValidateAndAdjust(t *testing.T) { + t.Parallel() + conf := GetDefaultServerConfig().Clone().Sorter + + require.Nil(t, conf.ValidateAndAdjust()) + conf.LevelDB.Compression = "none" + require.Nil(t, conf.ValidateAndAdjust()) + conf.LevelDB.Compression = "snappy" + require.Nil(t, conf.ValidateAndAdjust()) + conf.LevelDB.Compression = "invalid" + require.Error(t, conf.ValidateAndAdjust()) + conf.LevelDB.CleanupSpeedLimit = 0 + require.Error(t, conf.ValidateAndAdjust()) +} diff --git a/pkg/config/sorter.go b/pkg/config/sorter.go index 7ddef8b9218..9035fb66420 100644 --- a/pkg/config/sorter.go +++ b/pkg/config/sorter.go @@ -29,27 +29,96 @@ type SorterConfig struct { NumWorkerPoolGoroutine int `toml:"num-workerpool-goroutine" json:"num-workerpool-goroutine"` // the directory used to store the temporary files generated by the sorter SortDir string `toml:"sort-dir" json:"sort-dir"` + + // EnableLevelDB enables leveldb sorter. + // + // The default value is true. + // TODO: turn on after GA. + EnableLevelDB bool `toml:"enable-leveldb-sorter" json:"enable-leveldb-sorter"` + LevelDB LevelDBConfig `toml:"leveldb" json:"leveldb"` +} + +// LevelDBConfig represents leveldb sorter config. +type LevelDBConfig struct { + // LevelDBCount is the number of leveldb count. + // + // The default value is 16. + LevelDBCount int `toml:"leveldb-count" json:"leveldb-count"` + // LevelDBConcurrency is the maximum write and read concurrency. + // + // The default value is 256. + LevelDBConcurrency int `toml:"leveldb-concurrency" json:"leveldb-concurrency"` + // MaxOpenFiles is the maximum number of open FD by leveldb sorter. + // + // The default value is 10000. + MaxOpenFiles int `toml:"max-open-files" json:"max-open-files"` + // BlockSize the block size of leveldb sorter. + // + // The default value is 65536, 64KB. + BlockSize int `toml:"block-size" json:"block-size"` + // BlockCacheSize is the capacity of leveldb block cache. + // + // The default value is 0. + BlockCacheSize int `toml:"block-cache-size" json:"block-cache-size"` + // WriterBufferSize is the size of memory table of leveldb. + // + // The default value is 8388608, 8MiB. + WriterBufferSize int `toml:"writer-buffer-size" json:"writer-buffer-size"` + // Compression is the compression algorithm that is used by leveldb. + // Valid values are "none" or "snappy". + // + // The default value is "snappy". + Compression string `toml:"compression" json:"compression"` + // TargetFileSizeBase limits size of leveldb sst file that compaction generates. + // + // The default value is 8388608, 8MiB. + TargetFileSizeBase int `toml:"target-file-size-base" json:"target-file-size-base"` + // CompactionL0Trigger defines number of leveldb sst file at level-0 that will + // trigger compaction. + // + // The default value is 160. + CompactionL0Trigger int `toml:"compaction-l0-trigger" json:"compaction-l0-trigger"` + // WriteL0SlowdownTrigger defines number of leveldb sst file at level-0 that + // will trigger write slowdown. + // + // The default value is 1<<31 - 1. + WriteL0SlowdownTrigger int `toml:"write-l0-slowdown-trigger" json:"write-l0-slowdown-trigger"` + // WriteL0PauseTrigger defines number of leveldb sst file at level-0 that will + // pause write. + // + // The default value is 1<<31 - 1. + WriteL0PauseTrigger int `toml:"write-l0-pause-trigger" json:"write-l0-pause-trigger"` + // CleanupSpeedLimit limits clean up speed, based on key value entry count. + // + // The default value is 10000. + CleanupSpeedLimit int `toml:"cleanup-speed-limit" json:"cleanup-speed-limit"` } // ValidateAndAdjust validates and adjusts the sorter configuration func (c *SorterConfig) ValidateAndAdjust() error { if c.ChunkSizeLimit < 1*1024*1024 { - return cerror.ErrIllegalUnifiedSorterParameter.GenWithStackByArgs("chunk-size-limit should be at least 1MB") + return cerror.ErrIllegalSorterParameter.GenWithStackByArgs("chunk-size-limit should be at least 1MB") } if c.NumConcurrentWorker < 1 { - return cerror.ErrIllegalUnifiedSorterParameter.GenWithStackByArgs("num-concurrent-worker should be at least 1") + return cerror.ErrIllegalSorterParameter.GenWithStackByArgs("num-concurrent-worker should be at least 1") } if c.NumWorkerPoolGoroutine > 4096 { - return cerror.ErrIllegalUnifiedSorterParameter.GenWithStackByArgs("num-workerpool-goroutine should be at most 4096") + return cerror.ErrIllegalSorterParameter.GenWithStackByArgs("num-workerpool-goroutine should be at most 4096") } if c.NumConcurrentWorker > c.NumWorkerPoolGoroutine { - return cerror.ErrIllegalUnifiedSorterParameter.GenWithStackByArgs("num-concurrent-worker larger than num-workerpool-goroutine is useless") + return cerror.ErrIllegalSorterParameter.GenWithStackByArgs("num-concurrent-worker larger than num-workerpool-goroutine is useless") } if c.NumWorkerPoolGoroutine < 1 { - return cerror.ErrIllegalUnifiedSorterParameter.GenWithStackByArgs("num-workerpool-goroutine should be at least 1, larger than 8 is recommended") + return cerror.ErrIllegalSorterParameter.GenWithStackByArgs("num-workerpool-goroutine should be at least 1, larger than 8 is recommended") } if c.MaxMemoryPressure < 0 || c.MaxMemoryPressure > 100 { - return cerror.ErrIllegalUnifiedSorterParameter.GenWithStackByArgs("max-memory-percentage should be a percentage") + return cerror.ErrIllegalSorterParameter.GenWithStackByArgs("max-memory-percentage should be a percentage") + } + if c.LevelDB.Compression != "none" && c.LevelDB.Compression != "snappy" { + return cerror.ErrIllegalSorterParameter.GenWithStackByArgs("sorter.compression must be \"none\" or \"snappy\"") + } + if c.LevelDB.CleanupSpeedLimit <= 1 { + return cerror.ErrIllegalSorterParameter.GenWithStackByArgs("sorter.cleanup-speed-limit must be larger than 1") } return nil diff --git a/pkg/errors/errors.go b/pkg/errors/errors.go index 1d6abe860a6..780be7948fe 100644 --- a/pkg/errors/errors.go +++ b/pkg/errors/errors.go @@ -236,11 +236,11 @@ var ( ErrInvalidS3URI = errors.Normalize("invalid s3 uri: %s", errors.RFCCodeText("CDC:ErrInvalidS3URI")) ErrBufferLogTimeout = errors.Normalize("send row changed events to log buffer timeout", errors.RFCCodeText("CDC:ErrBufferLogTimeout")) - // unified sorter errors + // sorter errors ErrUnifiedSorterBackendTerminating = errors.Normalize("unified sorter backend is terminating", errors.RFCCodeText("CDC:ErrUnifiedSorterBackendTerminating")) - ErrIllegalUnifiedSorterParameter = errors.Normalize("illegal parameter for unified sorter: %s", errors.RFCCodeText("CDC:ErrIllegalUnifiedSorterParameter")) - ErrAsyncIOCancelled = errors.Normalize("asynchronous IO operation is cancelled. Internal use only, report a bug if seen in log", errors.RFCCodeText("CDC:ErrAsyncIOCancelled")) ErrUnifiedSorterIOError = errors.Normalize("unified sorter IO error. Make sure your sort-dir is configured correctly by passing a valid argument or toml file to `cdc server`, or if you use TiUP, review the settings in `tiup cluster edit-config`. Details: %s", errors.RFCCodeText("CDC:ErrUnifiedSorterIOError")) + ErrIllegalSorterParameter = errors.Normalize("illegal parameter for sorter: %s", errors.RFCCodeText("CDC:ErrIllegalSorterParameter")) + ErrAsyncIOCancelled = errors.Normalize("asynchronous IO operation is cancelled. Internal use only, report a bug if seen in log", errors.RFCCodeText("CDC:ErrAsyncIOCancelled")) ErrConflictingFileLocks = errors.Normalize("file lock conflict: %s", errors.RFCCodeText("ErrConflictingFileLocks")) ErrSortDirLockError = errors.Normalize("error encountered when locking sort-dir", errors.RFCCodeText("ErrSortDirLockError")) From 837508bbcbe4a9d95ede41c44f7cecafc9787078 Mon Sep 17 00:00:00 2001 From: Neil Shen Date: Sat, 6 Nov 2021 10:35:02 +0800 Subject: [PATCH 5/5] metrics: add changefeed checkepoint catch-up ETA (#3300) --- metrics/grafana/ticdc.json | 376 +++++++++++++++++++++++++------------ 1 file changed, 256 insertions(+), 120 deletions(-) diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index 33cbc23f028..73840c7a08d 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -125,7 +125,7 @@ "gnetId": null, "graphTooltip": 1, "id": null, - "iteration": 1633763775536, + "iteration": 1636101093950, "links": [], "panels": [ { @@ -1821,11 +1821,17 @@ }, { "aliasColors": {}, - "bars": false, + "bars": true, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of sink write duration of changefeeds", + "description": "Estimate the remaining time for a changefeed catch-up upstream TiDB.", + "fieldConfig": { + "defaults": { + "unit": "s" + }, + "overrides": [] + }, "fill": 1, "fillGradient": 0, "gridPos": { @@ -1835,11 +1841,13 @@ "y": 26 }, "hiddenSeries": false, - "id": 35, + "id": 253, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": true, + "hideZero": true, "max": false, "min": false, "rightSide": true, @@ -1847,12 +1855,12 @@ "total": false, "values": true }, - "lines": true, + "lines": false, "linewidth": 1, "links": [], "nullPointMode": "null", "options": { - "dataLinks": [] + "alertThreshold": true }, "paceLength": 10, "percentage": false, @@ -1865,32 +1873,20 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_sink_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "exemplar": true, + "expr": "abs(max(ticdc_owner_checkpoint_ts_lag{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"} / (deriv(ticdc_owner_checkpoint_ts{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])/1000)) by (changefeed))", "format": "time_series", + "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-p95", + "legendFormat": "{{changefeed}}", "refId": "A" - }, - { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p99", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p999", - "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Sink write duration percentile", + "title": "Changefeed checkpoint catch-up ETA", "tooltip": { "shared": true, "sort": 0, @@ -1907,10 +1903,10 @@ "yaxes": [ { "format": "s", - "label": null, + "label": "", "logBase": 2, - "max": null, - "min": null, + "max": "604800", + "min": "1", "show": true }, { @@ -2012,7 +2008,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of sink batch size", + "description": "Percentiles of sink write duration of changefeeds", "fill": 1, "fillGradient": 0, "gridPos": { @@ -2022,7 +2018,7 @@ "y": 33 }, "hiddenSeries": false, - "id": 36, + "id": 35, "legend": { "alignAsTable": true, "avg": false, @@ -2052,25 +2048,24 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.90, sum(rate(ticdc_sink_txn_batch_size_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "expr": "histogram_quantile(0.95, sum(rate(ticdc_sink_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance))", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}-p90", + "legendFormat": "{{instance}}-p95", "refId": "A" }, { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_txn_batch_size_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance))", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}-p99", + "legendFormat": "{{instance}}-p99", "refId": "B" }, { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_txn_batch_size_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance))", "format": "time_series", - "hide": true, "intervalFactor": 1, - "legendFormat": "{{capture}}-p999", + "legendFormat": "{{instance}}-p999", "refId": "C" } ], @@ -2078,7 +2073,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Sink write batch size percentile", + "title": "Sink write duration percentile", "tooltip": { "shared": true, "sort": 0, @@ -2094,7 +2089,7 @@ }, "yaxes": [ { - "format": "none", + "format": "s", "label": null, "logBase": 2, "max": null, @@ -2224,7 +2219,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of asynchronous flush sink duration of changefeeds", + "description": "Percentiles of sink batch size", "fill": 1, "fillGradient": 0, "gridPos": { @@ -2234,7 +2229,7 @@ "y": 40 }, "hiddenSeries": false, - "id": 98, + "id": 36, "legend": { "alignAsTable": true, "avg": false, @@ -2264,24 +2259,25 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_sink_flush_event_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance,type))", + "expr": "histogram_quantile(0.90, sum(rate(ticdc_sink_txn_batch_size_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}-p95", + "legendFormat": "{{capture}}-p90", "refId": "A" }, { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_flush_event_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance,type))", + "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_txn_batch_size_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}-p99", + "legendFormat": "{{capture}}-p99", "refId": "B" }, { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_flush_event_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance,type))", + "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_txn_batch_size_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", "format": "time_series", + "hide": true, "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}-p999", + "legendFormat": "{{capture}}-p999", "refId": "C" } ], @@ -2289,7 +2285,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Flush sink duration percentile", + "title": "Sink write batch size percentile", "tooltip": { "shared": true, "sort": 0, @@ -2305,7 +2301,7 @@ }, "yaxes": [ { - "format": "s", + "format": "none", "label": null, "logBase": 2, "max": null, @@ -2409,7 +2405,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Distribution of MySQL worker loads", + "description": "Percentiles of asynchronous flush sink duration of changefeeds", "fill": 1, "fillGradient": 0, "gridPos": { @@ -2419,17 +2415,15 @@ "y": 47 }, "hiddenSeries": false, - "id": 95, + "id": 98, "legend": { "alignAsTable": true, - "avg": true, + "avg": false, "current": true, - "max": true, + "max": false, "min": false, - "rightSide": false, + "rightSide": true, "show": true, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -2440,77 +2434,46 @@ "options": { "dataLinks": [] }, + "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, - "stack": true, + "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture,bucket)", + "expr": "histogram_quantile(0.95, sum(rate(ticdc_sink_flush_event_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance,type))", "format": "time_series", - "hide": true, - "interval": "", "intervalFactor": 1, - "legendFormat": "{{capture}}-{{bucket}}", + "legendFormat": "{{instance}}-{{type}}-p95", "refId": "A" }, { - "expr": "count(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) >= 0)", + "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_flush_event_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance,type))", "format": "time_series", - "hide": true, - "interval": "", "intervalFactor": 1, - "legendFormat": "total worker", + "legendFormat": "{{instance}}-{{type}}-p99", "refId": "B" }, { - "expr": "count(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 2)", + "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_flush_event_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance,type))", "format": "time_series", - "interval": "", "intervalFactor": 1, - "legendFormat": "0-2 row/s worker", + "legendFormat": "{{instance}}-{{type}}-p999", "refId": "C" - }, - { - "expr": "count(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 2 and rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 10)", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": "2-10 row/s worker", - "refId": "D" - }, - { - "expr": "count(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 10 and rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 100)", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": "10-100 row/s worker", - "refId": "E" - }, - { - "expr": "count(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 100)", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": ">100 row/s worker", - "refId": "F" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "MySQL sink worker load", + "title": "Flush sink duration percentile", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -2523,9 +2486,9 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, - "logBase": 1, + "logBase": 2, "max": null, "min": null, "show": true @@ -2628,7 +2591,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of detecting and waiting conflict duration of MySQL sink", + "description": "Distribution of MySQL worker loads", "fill": 1, "fillGradient": 0, "gridPos": { @@ -2638,15 +2601,17 @@ "y": 54 }, "hiddenSeries": false, - "id": 83, + "id": 95, "legend": { "alignAsTable": true, - "avg": false, + "avg": true, "current": true, "max": true, "min": false, "rightSide": false, "show": true, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -2663,39 +2628,68 @@ "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, - "stack": false, + "stack": true, "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.95,sum(rate(ticdc_sink_conflict_detect_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", + "expr": "sum(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture,bucket)", "format": "time_series", + "hide": true, "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-p95", + "legendFormat": "{{capture}}-{{bucket}}", "refId": "A" }, { - "expr": "histogram_quantile(0.99,sum(rate(ticdc_sink_conflict_detect_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", + "expr": "count(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) >= 0)", "format": "time_series", + "hide": true, "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-p99", + "legendFormat": "total worker", "refId": "B" }, { - "expr": "histogram_quantile(0.999,sum(rate(ticdc_sink_conflict_detect_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", + "expr": "count(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 2)", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-p999", + "legendFormat": "0-2 row/s worker", "refId": "C" + }, + { + "expr": "count(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 2 and rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 10)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "2-10 row/s worker", + "refId": "D" + }, + { + "expr": "count(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 10 and rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 100)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "10-100 row/s worker", + "refId": "E" + }, + { + "expr": "count(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 100)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": ">100 row/s worker", + "refId": "F" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "MySQL sink conflict detect duration percentile", + "title": "MySQL sink worker load", "tooltip": { "shared": true, "sort": 2, @@ -2711,9 +2705,9 @@ }, "yaxes": [ { - "format": "s", + "format": "short", "label": null, - "logBase": 2, + "logBase": 1, "max": null, "min": null, "show": true @@ -2816,7 +2810,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of executing DDL duration of MySQL sink", + "description": "Percentiles of detecting and waiting conflict duration of MySQL sink", "fill": 1, "fillGradient": 0, "gridPos": { @@ -2826,7 +2820,7 @@ "y": 61 }, "hiddenSeries": false, - "id": 166, + "id": 83, "legend": { "alignAsTable": true, "avg": false, @@ -2855,7 +2849,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.95,sum(rate(ticdc_sink_ddl_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", + "expr": "histogram_quantile(0.95,sum(rate(ticdc_sink_conflict_detect_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", "format": "time_series", "interval": "", "intervalFactor": 1, @@ -2863,7 +2857,7 @@ "refId": "A" }, { - "expr": "histogram_quantile(0.99,sum(rate(ticdc_sink_ddl_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", + "expr": "histogram_quantile(0.99,sum(rate(ticdc_sink_conflict_detect_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", "format": "time_series", "interval": "", "intervalFactor": 1, @@ -2871,7 +2865,7 @@ "refId": "B" }, { - "expr": "histogram_quantile(0.999,sum(rate(ticdc_sink_ddl_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", + "expr": "histogram_quantile(0.999,sum(rate(ticdc_sink_conflict_detect_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", "format": "time_series", "interval": "", "intervalFactor": 1, @@ -2883,7 +2877,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "MySQL sink ddl execution duration percentile", + "title": "MySQL sink conflict detect duration percentile", "tooltip": { "shared": true, "sort": 2, @@ -3011,13 +3005,123 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "Percentiles of executing DDL duration of MySQL sink", "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, "y": 68 }, + "hiddenSeries": false, + "id": 166, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.95,sum(rate(ticdc_sink_ddl_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p95", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.99,sum(rate(ticdc_sink_ddl_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p99", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.999,sum(rate(ticdc_sink_ddl_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p999", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "MySQL sink ddl execution duration percentile", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 2, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fill": 1, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 75 + }, "id": 164, "legend": { "avg": false, @@ -3091,6 +3195,7 @@ } }, { + "columns": [], "datasource": "${DS_TEST-CLUSTER}", "fieldConfig": { "defaults": { @@ -3144,11 +3249,12 @@ } ] }, + "fontSize": "100%", "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 69 + "y": 76 }, "id": 200, "options": { @@ -3160,7 +3266,36 @@ } ] }, + "pageSize": null, "pluginVersion": "6.1.6", + "scroll": true, + "showHeader": true, + "sort": { + "col": 0, + "desc": true + }, + "styles": [ + { + "alias": "Time", + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "pattern": "Time", + "type": "date" + }, + { + "alias": "", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "decimals": 2, + "pattern": "/.*/", + "thresholds": [], + "type": "number", + "unit": "short" + } + ], "targets": [ { "exemplar": true, @@ -3174,6 +3309,7 @@ } ], "title": "Schema Storage GC progress", + "transform": "table", "transformations": [ { "id": "filterFieldsByName", @@ -9545,5 +9681,5 @@ "timezone": "browser", "title": "Test-Cluster-TiCDC", "uid": "YiGL8hBZ1", - "version": 24 -} + "version": 25 +} \ No newline at end of file