From 3c797fdc64b82a8c7ec6897df2836a1324f26735 Mon Sep 17 00:00:00 2001 From: Calvin Neo Date: Tue, 25 Jan 2022 16:56:11 +0800 Subject: [PATCH] *: fix data races in from MockTiFlash (#31836) close pingcap/tidb#31830 --- ddl/ddl_tiflash_test.go | 80 +++++-------------- domain/infosync/tiflash_manager.go | 123 +++++++++++++++++++++++++++-- 2 files changed, 138 insertions(+), 65 deletions(-) diff --git a/ddl/ddl_tiflash_test.go b/ddl/ddl_tiflash_test.go index 47eb445c2251a..3a7f299ea8f66 100644 --- a/ddl/ddl_tiflash_test.go +++ b/ddl/ddl_tiflash_test.go @@ -96,50 +96,15 @@ func (s *tiflashDDLTestSuite) SetUpSuite(c *C) { } func (s *tiflashDDLTestSuite) TearDownSuite(c *C) { + s.tiflash.Lock() s.tiflash.StatusServer.Close() + s.tiflash.Unlock() s.dom.Close() err := s.store.Close() c.Assert(err, IsNil) ddl.PollTiFlashInterval = 2 * time.Second } -// Compare supposed rule, and we actually get from TableInfo -func isRuleMatch(rule placement.TiFlashRule, startKey string, endKey string, count int, labels []string) bool { - // Compute startKey - if rule.StartKeyHex == startKey && rule.EndKeyHex == endKey { - ok := false - for _, c := range rule.Constraints { - if c.Key == "engine" && len(c.Values) == 1 && c.Values[0] == "tiflash" && c.Op == placement.In { - ok = true - break - } - } - if !ok { - return false - } - - if len(rule.LocationLabels) == len(labels) { - for i, lb := range labels { - if lb != rule.LocationLabels[i] { - return false - } - } - } else { - return false - } - - if rule.Count != count { - return false - } - if rule.Role != placement.Learner { - return false - } - } else { - return false - } - return true -} - func ChangeGCSafePoint(tk *testkit.TestKit, t time.Time, enable string, lifeTime string) { gcTimeFormat := "20060102-15:04:05 -0700 MST" lastSafePoint := t.Format(gcTimeFormat) @@ -161,12 +126,7 @@ func ChangeGCSafePoint(tk *testkit.TestKit, t time.Time, enable string, lifeTime } func (s *tiflashDDLTestSuite) CheckPlacementRule(rule placement.TiFlashRule) bool { - for _, r := range s.tiflash.GlobalTiFlashPlacementRules { - if isRuleMatch(rule, r.StartKeyHex, r.EndKeyHex, r.Count, r.LocationLabels) { - return true - } - } - return false + return s.tiflash.CheckPlacementRule(rule) } func (s *tiflashDDLTestSuite) CheckFlashback(tk *testkit.TestKit, c *C) { @@ -190,12 +150,12 @@ func (s *tiflashDDLTestSuite) CheckFlashback(tk *testkit.TestKit, c *C) { if tb.Meta().Partition != nil { for _, e := range tb.Meta().Partition.Definitions { ruleName := fmt.Sprintf("table-%v-r", e.ID) - _, ok := s.tiflash.GlobalTiFlashPlacementRules[ruleName] + _, ok := s.tiflash.GetPlacementRule(ruleName) c.Assert(ok, Equals, true) } } else { ruleName := fmt.Sprintf("table-%v-r", tb.Meta().ID) - _, ok := s.tiflash.GlobalTiFlashPlacementRules[ruleName] + _, ok := s.tiflash.GetPlacementRule(ruleName) c.Assert(ok, Equals, true) } } @@ -247,7 +207,7 @@ func (s *tiflashDDLTestSuite) TestTiFlashNoRedundantPDRules(c *C) { defer fCancelPD() // Clean all rules - s.tiflash.GlobalTiFlashPlacementRules = make(map[string]placement.TiFlashRule) + s.tiflash.CleanPlacementRules() tk.MustExec("use test") tk.MustExec("drop table if exists ddltiflash") tk.MustExec("drop table if exists ddltiflashp") @@ -255,56 +215,56 @@ func (s *tiflashDDLTestSuite) TestTiFlashNoRedundantPDRules(c *C) { tk.MustExec("create table ddltiflashp(z int) PARTITION BY RANGE(z) (PARTITION p0 VALUES LESS THAN (10),PARTITION p1 VALUES LESS THAN (20), PARTITION p2 VALUES LESS THAN (30))") total := 0 - c.Assert(len(s.tiflash.GlobalTiFlashPlacementRules), Equals, total) + c.Assert(s.tiflash.PlacementRulesLen(), Equals, total) tk.MustExec("alter table ddltiflash set tiflash replica 1") total += 1 time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailable) - c.Assert(len(s.tiflash.GlobalTiFlashPlacementRules), Equals, total) + c.Assert(s.tiflash.PlacementRulesLen(), Equals, total) tk.MustExec("alter table ddltiflashp set tiflash replica 1") total += 3 time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailablePartitionTable) - c.Assert(len(s.tiflash.GlobalTiFlashPlacementRules), Equals, total) + c.Assert(s.tiflash.PlacementRulesLen(), Equals, total) lessThan := 40 tk.MustExec(fmt.Sprintf("ALTER TABLE ddltiflashp ADD PARTITION (PARTITION pn VALUES LESS THAN (%v))", lessThan)) total += 1 time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailablePartitionTable) - c.Assert(len(s.tiflash.GlobalTiFlashPlacementRules), Equals, total) + c.Assert(s.tiflash.PlacementRulesLen(), Equals, total) tk.MustExec("alter table ddltiflashp truncate partition p1") total += 1 time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailablePartitionTable) - c.Assert(len(s.tiflash.GlobalTiFlashPlacementRules), Equals, total) + c.Assert(s.tiflash.PlacementRulesLen(), Equals, total) // Now gc will trigger, and will remove dropped partition. c.Assert(gcWorker.DeleteRanges(context.TODO(), math.MaxInt64), IsNil) total -= 1 time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailablePartitionTable) - c.Assert(len(s.tiflash.GlobalTiFlashPlacementRules), Equals, total) + c.Assert(s.tiflash.PlacementRulesLen(), Equals, total) tk.MustExec("alter table ddltiflashp drop partition p2") c.Assert(gcWorker.DeleteRanges(context.TODO(), math.MaxInt64), IsNil) total -= 1 time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailablePartitionTable) - c.Assert(len(s.tiflash.GlobalTiFlashPlacementRules), Equals, total) + c.Assert(s.tiflash.PlacementRulesLen(), Equals, total) tk.MustExec("truncate table ddltiflash") total += 1 time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailablePartitionTable) - c.Assert(len(s.tiflash.GlobalTiFlashPlacementRules), Equals, total) + c.Assert(s.tiflash.PlacementRulesLen(), Equals, total) c.Assert(gcWorker.DeleteRanges(context.TODO(), math.MaxInt64), IsNil) total -= 1 time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailablePartitionTable) - c.Assert(len(s.tiflash.GlobalTiFlashPlacementRules), Equals, total) + c.Assert(s.tiflash.PlacementRulesLen(), Equals, total) tk.MustExec("drop table ddltiflash") total -= 1 time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailablePartitionTable) c.Assert(gcWorker.DeleteRanges(context.TODO(), math.MaxInt64), IsNil) - c.Assert(len(s.tiflash.GlobalTiFlashPlacementRules), Equals, total) + c.Assert(s.tiflash.PlacementRulesLen(), Equals, total) } func (s *tiflashDDLTestSuite) TestTiFlashReplicaPartitionTableNormal(c *C) { @@ -335,7 +295,7 @@ func (s *tiflashDDLTestSuite) TestTiFlashReplicaPartitionTableNormal(c *C) { for _, p := range pi.Definitions { c.Assert(tb2.Meta().TiFlashReplica.IsPartitionAvailable(p.ID), Equals, true) if len(p.LessThan) == 1 && p.LessThan[0] == lessThan { - table, ok := s.tiflash.SyncStatus[int(p.ID)] + table, ok := s.tiflash.GetTableSyncStatus(int(p.ID)) c.Assert(ok, Equals, true) c.Assert(table.Accel, Equals, true) } @@ -373,7 +333,7 @@ func (s *tiflashDDLTestSuite) TestTiFlashReplicaPartitionTableBlock(c *C) { for _, p := range pi.Definitions { c.Assert(tb.Meta().TiFlashReplica.IsPartitionAvailable(p.ID), Equals, true) if len(p.LessThan) == 1 && p.LessThan[0] == lessThan { - table, ok := s.tiflash.SyncStatus[int(p.ID)] + table, ok := s.tiflash.GetTableSyncStatus(int(p.ID)) c.Assert(ok, Equals, true) c.Assert(table.Accel, Equals, true) } @@ -589,9 +549,9 @@ func (s *tiflashDDLTestSuite) TestSetPlacementRuleFail(c *C) { tk.MustExec("use test") tk.MustExec("drop table if exists ddltiflash") tk.MustExec("create table ddltiflash(z int)") - s.tiflash.PdEnabled = false + s.tiflash.PdSwitch(false) defer func() { - s.tiflash.PdEnabled = true + s.tiflash.PdSwitch(true) }() tk.MustExec("alter table ddltiflash set tiflash replica 1") tb, err := s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("ddltiflash")) diff --git a/domain/infosync/tiflash_manager.go b/domain/infosync/tiflash_manager.go index d04db0e28ceb3..43ee048ca628b 100644 --- a/domain/infosync/tiflash_manager.go +++ b/domain/infosync/tiflash_manager.go @@ -184,6 +184,8 @@ func (m *TiFlashPDPlacementManager) GetStoresStat(ctx context.Context) (*helper. type mockTiFlashPlacementManager struct { sync.Mutex + // Set to nil if there is no need to set up a mock TiFlash server. + // Otherwise use NewMockTiFlash to create one. tiflash *MockTiFlash } @@ -241,6 +243,7 @@ func (m *mockTiFlashTableInfo) String() string { // MockTiFlash mocks a TiFlash, with necessary Pd support. type MockTiFlash struct { + sync.Mutex StatusAddr string StatusServer *httptest.Server SyncStatus map[int]mockTiFlashTableInfo @@ -250,7 +253,9 @@ type MockTiFlash struct { StartTime time.Time } -func (tiflash *MockTiFlash) setUpMockTiFlashHTTPServer() (*httptest.Server, string) { +func (tiflash *MockTiFlash) setUpMockTiFlashHTTPServer() { + tiflash.Lock() + defer tiflash.Unlock() // mock TiFlash http server router := mux.NewRouter() server := httptest.NewServer(router) @@ -259,6 +264,8 @@ func (tiflash *MockTiFlash) setUpMockTiFlashHTTPServer() (*httptest.Server, stri statusAddrVec := strings.Split(statusAddr, ":") statusPort, _ := strconv.Atoi(statusAddrVec[1]) router.HandleFunc("/tiflash/sync-status/{tableid:\\d+}", func(w http.ResponseWriter, req *http.Request) { + tiflash.Lock() + defer tiflash.Unlock() params := mux.Vars(req) tableID, err := strconv.Atoi(params["tableid"]) if err != nil { @@ -276,11 +283,14 @@ func (tiflash *MockTiFlash) setUpMockTiFlashHTTPServer() (*httptest.Server, stri _, _ = w.Write([]byte(table.String())) }) router.HandleFunc("/config", func(w http.ResponseWriter, req *http.Request) { + tiflash.Lock() + defer tiflash.Unlock() s := fmt.Sprintf("{\n \"engine-store\": {\n \"http_port\": %v\n }\n}", statusPort) w.WriteHeader(http.StatusOK) _, _ = w.Write([]byte(s)) }) - return server, statusAddr + tiflash.StatusServer = server + tiflash.StatusAddr = statusAddr } // NewMockTiFlash creates a MockTiFlash with a mocked TiFlash server. @@ -294,14 +304,14 @@ func NewMockTiFlash() *MockTiFlash { TiflashDelay: 0, StartTime: time.Now(), } - server, addr := tiflash.setUpMockTiFlashHTTPServer() - tiflash.StatusAddr = addr - tiflash.StatusServer = server + tiflash.setUpMockTiFlashHTTPServer() return tiflash } // HandleSetPlacementRule is mock function for SetTiFlashPlacementRule. func (tiflash *MockTiFlash) HandleSetPlacementRule(rule placement.TiFlashRule) error { + tiflash.Lock() + defer tiflash.Unlock() if !tiflash.PdEnabled { return errors.New("pd server is manually disabled, just quit") } @@ -339,11 +349,15 @@ func (tiflash *MockTiFlash) HandleSetPlacementRule(rule placement.TiFlashRule) e // HandleDeletePlacementRule is mock function for DeleteTiFlashPlacementRule. func (tiflash *MockTiFlash) HandleDeletePlacementRule(group string, ruleID string) { + tiflash.Lock() + defer tiflash.Unlock() delete(tiflash.GlobalTiFlashPlacementRules, ruleID) } // HandleGetGroupRules is mock function for GetTiFlashGroupRules. func (tiflash *MockTiFlash) HandleGetGroupRules(group string) ([]placement.TiFlashRule, error) { + tiflash.Lock() + defer tiflash.Unlock() var result = make([]placement.TiFlashRule, 0) for _, item := range tiflash.GlobalTiFlashPlacementRules { result = append(result, item) @@ -353,6 +367,8 @@ func (tiflash *MockTiFlash) HandleGetGroupRules(group string) ([]placement.TiFla // HandlePostAccelerateSchedule is mock function for PostAccelerateSchedule func (tiflash *MockTiFlash) HandlePostAccelerateSchedule(endKey string) error { + tiflash.Lock() + defer tiflash.Unlock() tableID := helper.GetTiFlashTableIDFromEndKey(endKey) table, ok := tiflash.SyncStatus[int(tableID)] @@ -384,6 +400,8 @@ func (tiflash *MockTiFlash) HandleGetPDRegionRecordStats(_ int64) helper.PDRegio // HandleGetStoresStat is mock function for GetStoresStat. // It returns address of our mocked TiFlash server. func (tiflash *MockTiFlash) HandleGetStoresStat() *helper.StoresStat { + tiflash.Lock() + defer tiflash.Unlock() return &helper.StoresStat{ Count: 1, Stores: []helper.StoreStat{ @@ -407,6 +425,98 @@ func (tiflash *MockTiFlash) HandleGetStoresStat() *helper.StoresStat { } } +// Compare supposed rule, and we actually get from TableInfo +func isRuleMatch(rule placement.TiFlashRule, startKey string, endKey string, count int, labels []string) bool { + // Compute startKey + if rule.StartKeyHex == startKey && rule.EndKeyHex == endKey { + ok := false + for _, c := range rule.Constraints { + if c.Key == "engine" && len(c.Values) == 1 && c.Values[0] == "tiflash" && c.Op == placement.In { + ok = true + break + } + } + if !ok { + return false + } + + if len(rule.LocationLabels) == len(labels) { + for i, lb := range labels { + if lb != rule.LocationLabels[i] { + return false + } + } + } else { + return false + } + + if rule.Count != count { + return false + } + if rule.Role != placement.Learner { + return false + } + } else { + return false + } + return true +} + +// CheckPlacementRule find if a given rule precisely matches already set rules. +func (tiflash *MockTiFlash) CheckPlacementRule(rule placement.TiFlashRule) bool { + tiflash.Lock() + defer tiflash.Unlock() + for _, r := range tiflash.GlobalTiFlashPlacementRules { + if isRuleMatch(rule, r.StartKeyHex, r.EndKeyHex, r.Count, r.LocationLabels) { + return true + } + } + return false +} + +// GetPlacementRule find a rule by name. +func (tiflash *MockTiFlash) GetPlacementRule(ruleName string) (*placement.TiFlashRule, bool) { + tiflash.Lock() + defer tiflash.Unlock() + if r, ok := tiflash.GlobalTiFlashPlacementRules[ruleName]; ok { + p := r + return &p, ok + } + return nil, false +} + +// CleanPlacementRules cleans all placement rules. +func (tiflash *MockTiFlash) CleanPlacementRules() { + tiflash.Lock() + defer tiflash.Unlock() + tiflash.GlobalTiFlashPlacementRules = make(map[string]placement.TiFlashRule) +} + +// PlacementRulesLen gets length of all currently set placement rules. +func (tiflash *MockTiFlash) PlacementRulesLen() int { + tiflash.Lock() + defer tiflash.Unlock() + return len(tiflash.GlobalTiFlashPlacementRules) +} + +// GetTableSyncStatus returns table sync status by given tableID. +func (tiflash *MockTiFlash) GetTableSyncStatus(tableID int) (*mockTiFlashTableInfo, bool) { + tiflash.Lock() + defer tiflash.Unlock() + if r, ok := tiflash.SyncStatus[tableID]; ok { + p := r + return &p, ok + } + return nil, false +} + +// PdSwitch controls if pd is enabled. +func (tiflash *MockTiFlash) PdSwitch(enabled bool) { + tiflash.Lock() + defer tiflash.Unlock() + tiflash.PdEnabled = enabled +} + // SetPlacementRule is a helper function to set placement rule. func (m *mockTiFlashPlacementManager) SetPlacementRule(ctx context.Context, rule placement.TiFlashRule) error { m.Lock() @@ -476,6 +586,9 @@ func (m *mockTiFlashPlacementManager) GetStoresStat(ctx context.Context) (*helpe func (m *mockTiFlashPlacementManager) Close(ctx context.Context) { m.Lock() defer m.Unlock() + if m.tiflash == nil { + return + } if m.tiflash.StatusServer != nil { m.tiflash.StatusServer.Close() }