diff --git a/.github/dependabot.yml b/.github/dependabot.yml new file mode 100644 index 00000000000..6fcbe6d6bd6 --- /dev/null +++ b/.github/dependabot.yml @@ -0,0 +1,14 @@ +version: 2 +updates: + - package-ecosystem: "npm" + directory: "/dm/ui" + schedule: + interval: "weekly" + # Specify labels for npm pull requests. + labels: + - "dependencies" + - "javascript" + - "release-note-none" + - "skip-issue-check" + # We use this option to make these configurations valid only for PRs with security issues. + open-pull-requests-limit: 0 diff --git a/Makefile b/Makefile index 2b12c40d678..c338ba4dac1 100644 --- a/Makefile +++ b/Makefile @@ -277,13 +277,22 @@ dm_generate_openapi: tools/bin/oapi-codegen cd dm && ../tools/bin/oapi-codegen --config=openapi/spec/types-gen-cfg.yaml openapi/spec/dm.yaml cd dm && ../tools/bin/oapi-codegen --config=openapi/spec/client-gen-cfg.yaml openapi/spec/dm.yaml -dm_unit_test: check_failpoint_ctl +define run_dm_unit_test + @echo "running unit test for packages:" $(1) mkdir -p $(DM_TEST_DIR) $(FAILPOINT_ENABLE) @export log_level=error; \ - $(GOTEST) -timeout 5m -covermode=atomic -coverprofile="$(DM_TEST_DIR)/cov.unit_test.out" $(DM_PACKAGES) \ + $(GOTEST) -timeout 5m -covermode=atomic -coverprofile="$(DM_TEST_DIR)/cov.unit_test.out" $(1) \ || { $(FAILPOINT_DISABLE); exit 1; } $(FAILPOINT_DISABLE) +endef + +dm_unit_test: check_failpoint_ctl + $(call run_dm_unit_test,$(DM_PACKAGES)) + +# run unit test for the specified pkg only, like `make dm_unit_test_pkg PKG=github.com/pingcap/tiflow/dm/dm/master` +dm_unit_test_pkg: check_failpoint_ctl + $(call run_dm_unit_test,$(PKG)) dm_unit_test_in_verify_ci: check_failpoint_ctl tools/bin/gotestsum tools/bin/gocov tools/bin/gocov-xml mkdir -p $(DM_TEST_DIR) diff --git a/cdc/api/middleware.go b/cdc/api/middleware.go new file mode 100644 index 00000000000..0abe7c38cf0 --- /dev/null +++ b/cdc/api/middleware.go @@ -0,0 +1,72 @@ +// Copyright 2022 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 api + +import ( + "net/http" + "time" + + "github.com/gin-gonic/gin" + "github.com/pingcap/log" + "github.com/pingcap/tiflow/cdc/model" + "go.uber.org/zap" +) + +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("duration", 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 IsHTTPBadRequestError(err) { + c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) + } else { + c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + } + c.Abort() + return + } + } +} diff --git a/cdc/api/open.go b/cdc/api/open.go index 80041ffeca6..14f0b29c403 100644 --- a/cdc/api/open.go +++ b/cdc/api/open.go @@ -49,41 +49,62 @@ const ( // openAPI provides capture APIs. type openAPI struct { capture *capture.Capture + // use for unit test only + testStatusProvider owner.StatusProvider } -// RegisterOpoenAPIRoutes registers routes for OpenAPI -func RegisterOpoenAPIRoutes(router *gin.Engine, capture *capture.Capture) { - openAPI := openAPI{capture: capture} +func NewOpenAPI(c *capture.Capture) openAPI { + return openAPI{capture: c} +} + +// NewOpenAPI4Test return a openAPI for test +func NewOpenAPI4Test(c *capture.Capture, p owner.StatusProvider) openAPI { + return openAPI{capture: c, testStatusProvider: p} +} + +func (h *openAPI) statusProvider() owner.StatusProvider { + if h.testStatusProvider != nil { + return h.testStatusProvider + } + return h.capture.StatusProvider() +} + +// RegisterOpenAPIRoutes registers routes for OpenAPI +func RegisterOpenAPIRoutes(router *gin.Engine, api openAPI) { + v1 := router.Group("/api/v1") + + v1.Use(logMiddleware()) + v1.Use(errorHandleMiddleware()) // common API - router.GET("/api/v1/status", openAPI.ServerStatus) - router.GET("/api/v1/health", openAPI.Health) - router.POST("/api/v1/log", SetLogLevel) + v1.GET("/status", api.ServerStatus) + v1.GET("/health", api.Health) + v1.POST("/log", SetLogLevel) // changefeed API - changefeedGroup := router.Group("/api/v1/changefeeds") - changefeedGroup.GET("", openAPI.ListChangefeed) - changefeedGroup.GET("/:changefeed_id", openAPI.GetChangefeed) - changefeedGroup.POST("", openAPI.CreateChangefeed) - changefeedGroup.PUT("/:changefeed_id", openAPI.UpdateChangefeed) - changefeedGroup.POST("/:changefeed_id/pause", openAPI.PauseChangefeed) - changefeedGroup.POST("/:changefeed_id/resume", openAPI.ResumeChangefeed) - changefeedGroup.DELETE("/:changefeed_id", openAPI.RemoveChangefeed) - changefeedGroup.POST("/:changefeed_id/tables/rebalance_table", openAPI.RebalanceTable) - changefeedGroup.POST("/:changefeed_id/tables/move_table", openAPI.MoveTable) + changefeedGroup := v1.Group("/changefeeds") + changefeedGroup.GET("", api.ListChangefeed) + changefeedGroup.GET("/:changefeed_id", api.GetChangefeed) + changefeedGroup.POST("", api.CreateChangefeed) + changefeedGroup.PUT("/:changefeed_id", api.UpdateChangefeed) + changefeedGroup.POST("/:changefeed_id/pause", api.PauseChangefeed) + changefeedGroup.POST("/:changefeed_id/resume", api.ResumeChangefeed) + changefeedGroup.DELETE("/:changefeed_id", api.RemoveChangefeed) + changefeedGroup.POST("/:changefeed_id/tables/rebalance_table", api.RebalanceTable) + changefeedGroup.POST("/:changefeed_id/tables/move_table", api.MoveTable) // owner API - ownerGroup := router.Group("/api/v1/owner") - ownerGroup.POST("/resign", openAPI.ResignOwner) + ownerGroup := v1.Group("/owner") + ownerGroup.POST("/resign", api.ResignOwner) // processor API - processorGroup := router.Group("/api/v1/processors") - processorGroup.GET("", openAPI.ListProcessor) - processorGroup.GET("/:changefeed_id/:capture_id", openAPI.GetProcessor) + processorGroup := v1.Group("/processors") + processorGroup.GET("", api.ListProcessor) + processorGroup.GET("/:changefeed_id/:capture_id", api.GetProcessor) // capture API - captureGroup := router.Group("/api/v1/captures") - captureGroup.GET("", openAPI.ListCapture) + captureGroup := v1.Group("/captures") + captureGroup.GET("", api.ListCapture) } // ListChangefeed lists all changgefeeds in cdc cluster @@ -101,17 +122,17 @@ func (h *openAPI) ListChangefeed(c *gin.Context) { h.forwardToOwner(c) return } - statusProvider := h.capture.StatusProvider() + ctx := c.Request.Context() state := c.Query(apiOpVarChangefeedState) // get all changefeed status - statuses, err := statusProvider.GetAllChangeFeedStatuses(ctx) + statuses, err := h.statusProvider().GetAllChangeFeedStatuses(ctx) if err != nil { _ = c.Error(err) return } // get all changefeed infos - infos, err := statusProvider.GetAllChangeFeedInfo(ctx) + infos, err := h.statusProvider().GetAllChangeFeedInfo(ctx) if err != nil { // this call will return a parsedError generated by the error we passed in // so it is no need to check the parsedError @@ -166,7 +187,6 @@ func (h *openAPI) GetChangefeed(c *gin.Context) { h.forwardToOwner(c) return } - statusProvider := h.capture.StatusProvider() ctx := c.Request.Context() changefeedID := c.Param(apiOpVarChangefeedID) @@ -175,19 +195,19 @@ func (h *openAPI) GetChangefeed(c *gin.Context) { return } - info, err := statusProvider.GetChangeFeedInfo(ctx, changefeedID) + info, err := h.statusProvider().GetChangeFeedInfo(ctx, changefeedID) if err != nil { _ = c.Error(err) return } - status, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) + status, err := h.statusProvider().GetChangeFeedStatus(ctx, changefeedID) if err != nil { _ = c.Error(err) return } - processorInfos, err := statusProvider.GetAllTaskStatuses(ctx, changefeedID) + processorInfos, err := h.statusProvider().GetAllTaskStatuses(ctx, changefeedID) if err != nil { _ = c.Error(err) return @@ -279,7 +299,7 @@ func (h *openAPI) PauseChangefeed(c *gin.Context) { h.forwardToOwner(c) return } - statusProvider := h.capture.StatusProvider() + ctx := c.Request.Context() changefeedID := c.Param(apiOpVarChangefeedID) @@ -288,7 +308,7 @@ func (h *openAPI) PauseChangefeed(c *gin.Context) { return } // check if the changefeed exists - _, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) + _, err := h.statusProvider().GetChangeFeedStatus(ctx, changefeedID) if err != nil { _ = c.Error(err) return @@ -322,7 +342,7 @@ func (h *openAPI) ResumeChangefeed(c *gin.Context) { h.forwardToOwner(c) return } - statusProvider := h.capture.StatusProvider() + ctx := c.Request.Context() changefeedID := c.Param(apiOpVarChangefeedID) if err := model.ValidateChangefeedID(changefeedID); err != nil { @@ -330,7 +350,7 @@ func (h *openAPI) ResumeChangefeed(c *gin.Context) { return } // check if the changefeed exists - _, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) + _, err := h.statusProvider().GetChangeFeedStatus(ctx, changefeedID) if err != nil { _ = c.Error(err) return @@ -370,7 +390,7 @@ func (h *openAPI) UpdateChangefeed(c *gin.Context) { h.forwardToOwner(c) return } - statusProvider := h.capture.StatusProvider() + ctx := c.Request.Context() changefeedID := c.Param(apiOpVarChangefeedID) @@ -378,7 +398,7 @@ func (h *openAPI) UpdateChangefeed(c *gin.Context) { _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID)) return } - info, err := statusProvider.GetChangeFeedInfo(ctx, changefeedID) + info, err := h.statusProvider().GetChangeFeedInfo(ctx, changefeedID) if err != nil { _ = c.Error(err) return @@ -426,7 +446,7 @@ func (h *openAPI) RemoveChangefeed(c *gin.Context) { h.forwardToOwner(c) return } - statusProvider := h.capture.StatusProvider() + ctx := c.Request.Context() changefeedID := c.Param(apiOpVarChangefeedID) if err := model.ValidateChangefeedID(changefeedID); err != nil { @@ -434,7 +454,7 @@ func (h *openAPI) RemoveChangefeed(c *gin.Context) { return } // check if the changefeed exists - _, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) + _, err := h.statusProvider().GetChangeFeedStatus(ctx, changefeedID) if err != nil { _ = c.Error(err) return @@ -468,7 +488,7 @@ func (h *openAPI) RebalanceTable(c *gin.Context) { h.forwardToOwner(c) return } - statusProvider := h.capture.StatusProvider() + ctx := c.Request.Context() changefeedID := c.Param(apiOpVarChangefeedID) @@ -477,7 +497,7 @@ func (h *openAPI) RebalanceTable(c *gin.Context) { return } // check if the changefeed exists - _, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) + _, err := h.statusProvider().GetChangeFeedStatus(ctx, changefeedID) if err != nil { _ = c.Error(err) return @@ -508,7 +528,7 @@ func (h *openAPI) MoveTable(c *gin.Context) { h.forwardToOwner(c) return } - statusProvider := h.capture.StatusProvider() + ctx := c.Request.Context() changefeedID := c.Param(apiOpVarChangefeedID) if err := model.ValidateChangefeedID(changefeedID); err != nil { @@ -516,7 +536,7 @@ func (h *openAPI) MoveTable(c *gin.Context) { return } // check if the changefeed exists - _, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) + _, err := h.statusProvider().GetChangeFeedStatus(ctx, changefeedID) if err != nil { _ = c.Error(err) return @@ -582,7 +602,6 @@ func (h *openAPI) GetProcessor(c *gin.Context) { h.forwardToOwner(c) return } - statusProvider := h.capture.StatusProvider() ctx := c.Request.Context() @@ -598,7 +617,7 @@ func (h *openAPI) GetProcessor(c *gin.Context) { return } - statuses, err := statusProvider.GetAllTaskStatuses(ctx, changefeedID) + statuses, err := h.statusProvider().GetAllTaskStatuses(ctx, changefeedID) if err != nil { _ = c.Error(err) return @@ -609,7 +628,7 @@ func (h *openAPI) GetProcessor(c *gin.Context) { return } - positions, err := statusProvider.GetTaskPositions(ctx, changefeedID) + positions, err := h.statusProvider().GetTaskPositions(ctx, changefeedID) if err != nil { _ = c.Error(err) return @@ -648,10 +667,9 @@ func (h *openAPI) ListProcessor(c *gin.Context) { h.forwardToOwner(c) return } - statusProvider := h.capture.StatusProvider() ctx := c.Request.Context() - infos, err := statusProvider.GetProcessors(ctx) + infos, err := h.statusProvider().GetProcessors(ctx) if err != nil { _ = c.Error(err) return @@ -678,10 +696,9 @@ func (h *openAPI) ListCapture(c *gin.Context) { h.forwardToOwner(c) return } - statusProvider := h.capture.StatusProvider() ctx := c.Request.Context() - captureInfos, err := statusProvider.GetCaptures(ctx) + captureInfos, err := h.statusProvider().GetCaptures(ctx) if err != nil { _ = c.Error(err) return diff --git a/cdc/api/open_test.go b/cdc/api/open_test.go new file mode 100644 index 00000000000..48a1fb53d4d --- /dev/null +++ b/cdc/api/open_test.go @@ -0,0 +1,450 @@ +// 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 api + +import ( + "bytes" + "context" + "encoding/json" + "fmt" + "net/http" + "net/http/httptest" + "testing" + + "github.com/gin-gonic/gin" + "github.com/pingcap/log" + "github.com/pingcap/tiflow/cdc/capture" + "github.com/pingcap/tiflow/cdc/model" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + "go.uber.org/zap" +) + +const ( + changeFeedID = "test-changeFeed" + captureID = "test-capture" + nonExistChangefeedID = "non-exist-changefeed" +) + +type mockStatusProvider struct { + mock.Mock +} + +type testCase struct { + url string + method string +} + +func (p *mockStatusProvider) GetAllChangeFeedStatuses(ctx context.Context) (map[model.ChangeFeedID]*model.ChangeFeedStatus, error) { + args := p.Called(ctx) + return args.Get(0).(map[model.ChangeFeedID]*model.ChangeFeedStatus), args.Error(1) +} + +func (p *mockStatusProvider) GetChangeFeedStatus(ctx context.Context, changefeedID model.ChangeFeedID) (*model.ChangeFeedStatus, error) { + args := p.Called(ctx, changefeedID) + log.Info("err", zap.Error(args.Error(1))) + return args.Get(0).(*model.ChangeFeedStatus), args.Error(1) +} + +func (p *mockStatusProvider) GetAllChangeFeedInfo(ctx context.Context) (map[model.ChangeFeedID]*model.ChangeFeedInfo, error) { + args := p.Called(ctx) + return args.Get(0).(map[model.ChangeFeedID]*model.ChangeFeedInfo), args.Error(1) +} + +func (p *mockStatusProvider) GetChangeFeedInfo(ctx context.Context, changefeedID model.ChangeFeedID) (*model.ChangeFeedInfo, error) { + args := p.Called(ctx) + return args.Get(0).(*model.ChangeFeedInfo), args.Error(1) +} + +func (p *mockStatusProvider) GetAllTaskStatuses(ctx context.Context, changefeedID model.ChangeFeedID) (map[model.CaptureID]*model.TaskStatus, error) { + args := p.Called(ctx) + return args.Get(0).(map[model.CaptureID]*model.TaskStatus), args.Error(1) +} + +func (p *mockStatusProvider) GetTaskPositions(ctx context.Context, changefeedID model.ChangeFeedID) (map[model.CaptureID]*model.TaskPosition, error) { + args := p.Called(ctx) + return args.Get(0).(map[model.CaptureID]*model.TaskPosition), args.Error(1) +} + +func (p *mockStatusProvider) GetProcessors(ctx context.Context) ([]*model.ProcInfoSnap, error) { + args := p.Called(ctx) + return args.Get(0).([]*model.ProcInfoSnap), args.Error(1) +} + +func (p *mockStatusProvider) GetCaptures(ctx context.Context) ([]*model.CaptureInfo, error) { + args := p.Called(ctx) + return args.Get(0).([]*model.CaptureInfo), args.Error(1) +} + +func newRouter(p *mockStatusProvider) *gin.Engine { + c := capture.NewCapture4Test(true) + router := gin.New() + RegisterOpenAPIRoutes(router, NewOpenAPI4Test(c, p)) + return router +} + +func newStatusProvider() *mockStatusProvider { + statusProvider := &mockStatusProvider{} + statusProvider.On("GetChangeFeedStatus", mock.Anything, changeFeedID). + Return(&model.ChangeFeedStatus{CheckpointTs: 1}, nil) + + statusProvider.On("GetChangeFeedStatus", mock.Anything, nonExistChangefeedID). + Return(new(model.ChangeFeedStatus), + cerror.ErrChangeFeedNotExists.GenWithStackByArgs(nonExistChangefeedID)) + + statusProvider.On("GetAllTaskStatuses", mock.Anything). + Return(map[model.CaptureID]*model.TaskStatus{captureID: {}}, nil) + + statusProvider.On("GetTaskPositions", mock.Anything). + Return(map[model.CaptureID]*model.TaskPosition{captureID: {Error: &model.RunningError{Message: "test"}}}, nil) + + statusProvider.On("GetAllChangeFeedStatuses", mock.Anything). + Return(map[model.ChangeFeedID]*model.ChangeFeedStatus{ + changeFeedID + "1": {CheckpointTs: 1}, + changeFeedID + "2": {CheckpointTs: 2}, + }, nil) + + statusProvider.On("GetAllChangeFeedInfo", mock.Anything). + Return(map[model.ChangeFeedID]*model.ChangeFeedInfo{ + changeFeedID + "1": {State: model.StateNormal}, + changeFeedID + "2": {State: model.StateStopped}, + }, nil) + + statusProvider.On("GetAllTaskStatuses", mock.Anything). + Return(map[model.CaptureID]*model.TaskStatus{captureID: {}}, nil) + + statusProvider.On("GetChangeFeedInfo", mock.Anything). + Return(&model.ChangeFeedInfo{State: model.StateNormal}, nil) + + statusProvider.On("GetProcessors", mock.Anything). + Return([]*model.ProcInfoSnap{{CfID: changeFeedID, CaptureID: captureID}}, nil) + + statusProvider.On("GetCaptures", mock.Anything). + Return([]*model.CaptureInfo{{ID: captureID}}, nil) + + return statusProvider +} + +func TestListChangefeed(t *testing.T) { + t.Parallel() + router := newRouter(newStatusProvider()) + + // test list changefeed succeeded + api := testCase{url: "/api/v1/changefeeds", method: "GET"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 200, w.Code) + var resp []model.ChangefeedCommonInfo + err := json.NewDecoder(w.Body).Decode(&resp) + require.Nil(t, err) + require.Equal(t, 2, len(resp)) + + // test list changefeed with specific state + api = testCase{url: fmt.Sprintf("/api/v1/changefeeds?state=%s", "stopped"), method: "GET"} + w = httptest.NewRecorder() + req, _ = http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 200, w.Code) + resp = []model.ChangefeedCommonInfo{} + err = json.NewDecoder(w.Body).Decode(&resp) + require.Nil(t, err) + require.Equal(t, 1, len(resp)) + require.Equal(t, model.StateStopped, resp[0].FeedState) + require.Equal(t, uint64(0x2), resp[0].CheckpointTSO) +} + +func TestGetChangefeed(t *testing.T) { + t.Parallel() + router := newRouter(newStatusProvider()) + + // test get changefeed succeeded + api := testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s", changeFeedID), method: "GET"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 200, w.Code) + var resp model.ChangefeedDetail + err := json.NewDecoder(w.Body).Decode(&resp) + require.Nil(t, err) + require.Equal(t, model.StateNormal, resp.FeedState) + + // test get changefeed failed + api = testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s", nonExistChangefeedID), method: "GET"} + w = httptest.NewRecorder() + req, _ = http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 400, w.Code) + respErr := model.HTTPError{} + err = json.NewDecoder(w.Body).Decode(&respErr) + require.Nil(t, err) + require.Contains(t, respErr.Error, "changefeed not exists") +} + +func TestPauseChangefeed(t *testing.T) { + t.Parallel() + router := newRouter(newStatusProvider()) + // test pause changefeed succeeded + api := testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/pause", changeFeedID), method: "POST"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 202, w.Code) + + // test pause changefeed failed + api = testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/pause", nonExistChangefeedID), method: "POST"} + w = httptest.NewRecorder() + req, _ = http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 400, w.Code) + respErr := model.HTTPError{} + err := json.NewDecoder(w.Body).Decode(&respErr) + require.Nil(t, err) + require.Contains(t, respErr.Error, "changefeed not exists") +} + +func TestResumeChangefeed(t *testing.T) { + t.Parallel() + router := newRouter(newStatusProvider()) + // test resume changefeed succeeded + api := testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/resume", changeFeedID), method: "POST"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 202, w.Code) + + // test resume changefeed failed + api = testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/resume", nonExistChangefeedID), method: "POST"} + w = httptest.NewRecorder() + req, _ = http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 400, w.Code) + respErr := model.HTTPError{} + err := json.NewDecoder(w.Body).Decode(&respErr) + require.Nil(t, err) + require.Contains(t, respErr.Error, "changefeed not exists") +} + +func TestRemoveChangefeed(t *testing.T) { + t.Parallel() + router := newRouter(newStatusProvider()) + // test remove changefeed succeeded + api := testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s", changeFeedID), method: "DELETE"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 202, w.Code) + + // test remove changefeed failed + api = testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s", nonExistChangefeedID), method: "DELETE"} + w = httptest.NewRecorder() + req, _ = http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 400, w.Code) + respErr := model.HTTPError{} + err := json.NewDecoder(w.Body).Decode(&respErr) + require.Nil(t, err) + require.Contains(t, respErr.Error, "changefeed not exists") +} + +func TestRebalanceTable(t *testing.T) { + t.Parallel() + router := newRouter(newStatusProvider()) + // test rebalance table succeeded + api := testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/tables/rebalance_table", changeFeedID), method: "POST"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 202, w.Code) + + // test rebalance table failed + api = testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/tables/rebalance_table", nonExistChangefeedID), method: "POST"} + w = httptest.NewRecorder() + req, _ = http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 400, w.Code) + respErr := model.HTTPError{} + err := json.NewDecoder(w.Body).Decode(&respErr) + require.Nil(t, err) + require.Contains(t, respErr.Error, "changefeed not exists") +} + +func TestMoveTable(t *testing.T) { + t.Parallel() + + data := struct { + CaptureID string `json:"capture_id"` + TableID int64 `json:"table_id"` + }{captureID, 1} + b, err := json.Marshal(&data) + require.Nil(t, err) + body := bytes.NewReader(b) + + router := newRouter(newStatusProvider()) + // test move table succeeded + api := testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/tables/move_table", changeFeedID), method: "POST"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, body) + router.ServeHTTP(w, req) + require.Equal(t, 202, w.Code) + + // test move table failed + api = testCase{url: fmt.Sprintf("/api/v1/changefeeds/%s/tables/move_table", nonExistChangefeedID), method: "POST"} + w = httptest.NewRecorder() + req, _ = http.NewRequest(api.method, api.url, body) + router.ServeHTTP(w, req) + require.Equal(t, 400, w.Code) + respErr := model.HTTPError{} + err = json.NewDecoder(w.Body).Decode(&respErr) + require.Nil(t, err) + require.Contains(t, respErr.Error, "changefeed not exists") +} + +func TestResignOwner(t *testing.T) { + t.Parallel() + router := newRouter(newStatusProvider()) + // test resign owner succeeded + api := testCase{url: "/api/v1/owner/resign", method: "POST"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 202, w.Code) +} + +func TestGetProcessor(t *testing.T) { + t.Parallel() + router := newRouter(newStatusProvider()) + // test get processor succeeded + api := testCase{url: fmt.Sprintf("/api/v1/processors/%s/%s", changeFeedID, captureID), method: "GET"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 200, w.Code) + processorDetail := &model.ProcessorDetail{} + err := json.NewDecoder(w.Body).Decode(processorDetail) + require.Nil(t, err) + require.Equal(t, "test", processorDetail.Error.Message) + + // test get processor fail due to capture ID error + api = testCase{url: fmt.Sprintf("/api/v1/processors/%s/%s", changeFeedID, "non-exist-capture"), method: "GET"} + w = httptest.NewRecorder() + req, _ = http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 400, w.Code) + httpError := &model.HTTPError{} + err = json.NewDecoder(w.Body).Decode(httpError) + require.Nil(t, err) + require.Contains(t, httpError.Error, "capture not exists, key: non-exist-capture") +} + +func TestListProcessor(t *testing.T) { + t.Parallel() + router := newRouter(newStatusProvider()) + // test list processor succeeded + api := testCase{url: "/api/v1/processors", method: "GET"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 200, w.Code) + var resp []model.ProcessorCommonInfo + err := json.NewDecoder(w.Body).Decode(&resp) + require.Nil(t, err) + require.Equal(t, changeFeedID, resp[0].CfID) +} + +func TestListCapture(t *testing.T) { + t.Parallel() + router := newRouter(newStatusProvider()) + // test list processor succeeded + api := testCase{url: "/api/v1/captures", method: "GET"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 200, w.Code) + var resp []model.Capture + err := json.NewDecoder(w.Body).Decode(&resp) + require.Nil(t, err) + require.Equal(t, captureID, resp[0].ID) +} + +func TestServerStatus(t *testing.T) { + t.Parallel() + // capture is owner + ownerRouter := newRouter(newStatusProvider()) + api := testCase{url: "/api/v1/status", method: "GET"} + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + ownerRouter.ServeHTTP(w, req) + require.Equal(t, 200, w.Code) + var resp model.ServerStatus + err := json.NewDecoder(w.Body).Decode(&resp) + require.Nil(t, err) + require.Equal(t, "capture-for-test", resp.ID) + require.True(t, resp.IsOwner) + + // capture is not owner + c := capture.NewCapture4Test(false) + r := gin.New() + RegisterOpenAPIRoutes(r, NewOpenAPI4Test(c, nil)) + api = testCase{url: "/api/v1/status", method: "GET"} + w = httptest.NewRecorder() + req, _ = http.NewRequest(api.method, api.url, nil) + r.ServeHTTP(w, req) + require.Equal(t, 200, w.Code) + resp = model.ServerStatus{} + err = json.NewDecoder(w.Body).Decode(&resp) + require.Nil(t, err) + require.False(t, resp.IsOwner) +} + +func TestSetLogLevel(t *testing.T) { + t.Parallel() + + // test set log level succeeded + data := struct { + Level string `json:"log_level"` + }{"warn"} + router := newRouter(newStatusProvider()) + api := testCase{url: "/api/v1/log", method: "POST"} + w := httptest.NewRecorder() + b, err := json.Marshal(&data) + require.Nil(t, err) + body := bytes.NewReader(b) + req, _ := http.NewRequest(api.method, api.url, body) + router.ServeHTTP(w, req) + require.Equal(t, 200, w.Code) + + // test set log level failed + data = struct { + Level string `json:"log_level"` + }{"foo"} + api = testCase{url: "/api/v1/log", method: "POST"} + w = httptest.NewRecorder() + b, err = json.Marshal(&data) + require.Nil(t, err) + body = bytes.NewReader(b) + req, _ = http.NewRequest(api.method, api.url, body) + router.ServeHTTP(w, req) + require.Equal(t, 400, w.Code) + httpError := &model.HTTPError{} + err = json.NewDecoder(w.Body).Decode(httpError) + require.Nil(t, err) + require.Contains(t, httpError.Error, "fail to change log level: foo") +} + +// TODO: finished these test cases after we decouple those APIs from etcdClient. +func TestCreateChangefeed(t *testing.T) {} +func TestUpdateChangefeed(t *testing.T) {} +func TestHealth(t *testing.T) {} diff --git a/cdc/api/owner.go b/cdc/api/owner.go index d6f0726cdad..a8b565c4a1a 100644 --- a/cdc/api/owner.go +++ b/cdc/api/owner.go @@ -82,11 +82,16 @@ type ownerAPI struct { // RegisterOwnerAPIRoutes registers routes for owner APIs. func RegisterOwnerAPIRoutes(router *gin.Engine, capture *capture.Capture) { ownerAPI := ownerAPI{capture: capture} - router.POST("/capture/owner/resign", gin.WrapF(ownerAPI.handleResignOwner)) - router.POST("/capture/owner/admin", gin.WrapF(ownerAPI.handleChangefeedAdmin)) - router.POST("/capture/owner/rebalance_trigger", gin.WrapF(ownerAPI.handleRebalanceTrigger)) - router.POST("/capture/owner/move_table", gin.WrapF(ownerAPI.handleMoveTable)) - router.POST("/capture/owner/changefeed/query", gin.WrapF(ownerAPI.handleChangefeedQuery)) + owner := router.Group("/capture/owner") + + owner.Use(errorHandleMiddleware()) + owner.Use(logMiddleware()) + + owner.POST("/resign", gin.WrapF(ownerAPI.handleResignOwner)) + owner.POST("/admin", gin.WrapF(ownerAPI.handleChangefeedAdmin)) + owner.POST("/rebalance_trigger", gin.WrapF(ownerAPI.handleRebalanceTrigger)) + owner.POST("/move_table", gin.WrapF(ownerAPI.handleMoveTable)) + owner.POST("/changefeed/query", gin.WrapF(ownerAPI.handleChangefeedQuery)) } func handleOwnerResp(w http.ResponseWriter, err error) { @@ -269,7 +274,7 @@ func (h *ownerAPI) handleChangefeedQuery(w http.ResponseWriter, req *http.Reques writeData(w, resp) } -func handleAdminLogLevel(w http.ResponseWriter, r *http.Request) { +func HandleAdminLogLevel(w http.ResponseWriter, r *http.Request) { var level string data, err := io.ReadAll(r.Body) r.Body.Close() diff --git a/cdc/api/owner_test.go b/cdc/api/owner_test.go index 7c19890d191..bd09d0ec7cc 100644 --- a/cdc/api/owner_test.go +++ b/cdc/api/owner_test.go @@ -14,124 +14,64 @@ package api import ( - "bytes" "fmt" "io" "net/http" "net/http/httptest" "net/url" + "testing" "github.com/gin-gonic/gin" - "github.com/pingcap/check" - "github.com/pingcap/failpoint" - "github.com/pingcap/tiflow/cdc/capture" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" "go.etcd.io/etcd/clientv3/concurrency" ) -type httpStatusSuite struct{} - -var _ = check.Suite(&httpStatusSuite{}) - -func (s *httpStatusSuite) TestHTTPStatus(c *check.C) { - defer testleak.AfterTest(c)() - +func TestHTTPStatus(t *testing.T) { + t.Parallel() router := gin.New() - RegisterRoutes(router, capture.NewCapture4Test(), nil) - + RegisterOwnerAPIRoutes(router, nil) ts := httptest.NewServer(router) defer ts.Close() addr := ts.URL - testPprof(c, addr) - testReisgnOwner(c, addr) - testHandleChangefeedAdmin(c, addr) - testHandleRebalance(c, addr) - testHandleMoveTable(c, addr) - testHandleChangefeedQuery(c, addr) - testHandleFailpoint(c, addr) + testReisgnOwner(t, addr) + testHandleChangefeedAdmin(t, addr) + testHandleRebalance(t, addr) + testHandleMoveTable(t, addr) + testHandleChangefeedQuery(t, addr) } -func testPprof(c *check.C, addr string) { - testValidPprof := func(uri string) { - resp, err := http.Get(uri) - c.Assert(err, check.IsNil) - defer resp.Body.Close() - c.Assert(resp.StatusCode, check.Equals, 200) - _, err = io.ReadAll(resp.Body) - c.Assert(err, check.IsNil) - } - testValidPprof(fmt.Sprintf("%s/debug/pprof", addr)) - testValidPprof(fmt.Sprintf("%s/debug/pprof/cmdline", addr)) - testValidPprof(fmt.Sprintf("%s/debug/pprof/mutex", addr)) - testValidPprof(fmt.Sprintf("%s/debug/pprof/heap?debug=1", addr)) -} - -func testReisgnOwner(c *check.C, addr string) { +func testReisgnOwner(t *testing.T, addr string) { uri := fmt.Sprintf("%s/capture/owner/resign", addr) - testRequestNonOwnerFailed(c, uri) + testRequestNonOwnerFailed(t, uri) } -func testHandleChangefeedAdmin(c *check.C, addr string) { +func testHandleChangefeedAdmin(t *testing.T, addr string) { uri := fmt.Sprintf("%s/capture/owner/admin", addr) - testRequestNonOwnerFailed(c, uri) + testRequestNonOwnerFailed(t, uri) } -func testHandleRebalance(c *check.C, addr string) { +func testHandleRebalance(t *testing.T, addr string) { uri := fmt.Sprintf("%s/capture/owner/rebalance_trigger", addr) - testRequestNonOwnerFailed(c, uri) + testRequestNonOwnerFailed(t, uri) } -func testHandleMoveTable(c *check.C, addr string) { +func testHandleMoveTable(t *testing.T, addr string) { uri := fmt.Sprintf("%s/capture/owner/move_table", addr) - testRequestNonOwnerFailed(c, uri) + testRequestNonOwnerFailed(t, uri) } -func testHandleChangefeedQuery(c *check.C, addr string) { +func testHandleChangefeedQuery(t *testing.T, addr string) { uri := fmt.Sprintf("%s/capture/owner/changefeed/query", addr) - testRequestNonOwnerFailed(c, uri) + testRequestNonOwnerFailed(t, uri) } -func testRequestNonOwnerFailed(c *check.C, uri string) { +func testRequestNonOwnerFailed(t *testing.T, uri string) { resp, err := http.PostForm(uri, url.Values{}) - c.Assert(err, check.IsNil) + require.Nil(t, err) data, err := io.ReadAll(resp.Body) - c.Assert(err, check.IsNil) - defer resp.Body.Close() - c.Assert(resp.StatusCode, check.Equals, http.StatusBadRequest) - c.Assert(string(data), check.Equals, concurrency.ErrElectionNotLeader.Error()) -} - -func testHandleFailpoint(c *check.C, addr string) { - fp := "github.com/pingcap/tiflow/cdc/TestHandleFailpoint" - uri := fmt.Sprintf("%s/debug/fail/%s", addr, fp) - body := bytes.NewReader([]byte("return(true)")) - req, err := http.NewRequest("PUT", uri, body) - c.Assert(err, check.IsNil) - - resp, err := http.DefaultClient.Do(req) - c.Assert(err, check.IsNil) + require.Nil(t, err) defer resp.Body.Close() - c.Assert(resp.StatusCode, check.GreaterEqual, 200) - c.Assert(resp.StatusCode, check.Less, 300) - - failpointHit := false - failpoint.Inject("TestHandleFailpoint", func() { - failpointHit = true - }) - c.Assert(failpointHit, check.IsTrue) - - req, err = http.NewRequest("DELETE", uri, body) - c.Assert(err, check.IsNil) - resp, err = http.DefaultClient.Do(req) - c.Assert(err, check.IsNil) - defer resp.Body.Close() - c.Assert(resp.StatusCode, check.GreaterEqual, 200) - c.Assert(resp.StatusCode, check.Less, 300) - - failpointHit = false - failpoint.Inject("TestHandleFailpoint", func() { - failpointHit = true - }) - c.Assert(failpointHit, check.IsFalse) + require.Equal(t, http.StatusBadRequest, resp.StatusCode) + require.Equal(t, concurrency.ErrElectionNotLeader.Error(), string(data)) } diff --git a/cdc/api/router.go b/cdc/api/router.go deleted file mode 100644 index c12e5d356bf..00000000000 --- a/cdc/api/router.go +++ /dev/null @@ -1,73 +0,0 @@ -// 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 api - -import ( - "net/http" - "net/http/pprof" - - "github.com/gin-gonic/gin" - "github.com/pingcap/failpoint" - "github.com/pingcap/tiflow/cdc/capture" - "github.com/pingcap/tiflow/pkg/util" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promhttp" - swaggerFiles "github.com/swaggo/files" - ginSwagger "github.com/swaggo/gin-swagger" - - // use for OpenAPI online docs - _ "github.com/pingcap/tiflow/docs/swagger" -) - -// RegisterRoutes create a router for OpenAPI -func RegisterRoutes( - router *gin.Engine, - capture *capture.Capture, - registry prometheus.Gatherer, -) { - // online docs - router.GET("/swagger/*any", ginSwagger.WrapHandler(swaggerFiles.Handler)) - - // Open API - RegisterOpoenAPIRoutes(router, capture) - - // Owner API - RegisterOwnerAPIRoutes(router, capture) - - // Status API - RegisterStatusAPIRoutes(router, capture) - - // Log API - router.POST("/admin/log", gin.WrapF(handleAdminLogLevel)) - - // pprof debug API - pprofGroup := router.Group("/debug/pprof/") - pprofGroup.GET("", gin.WrapF(pprof.Index)) - pprofGroup.GET("/:any", gin.WrapF(pprof.Index)) - pprofGroup.GET("/cmdline", gin.WrapF(pprof.Cmdline)) - pprofGroup.GET("/profile", gin.WrapF(pprof.Profile)) - pprofGroup.GET("/symbol", gin.WrapF(pprof.Symbol)) - pprofGroup.GET("/trace", gin.WrapF(pprof.Trace)) - pprofGroup.GET("/threadcreate", gin.WrapF(pprof.Handler("threadcreate").ServeHTTP)) - - // Failpoint API - if util.FailpointBuild { - // `http.StripPrefix` is needed because `failpoint.HttpHandler` assumes that it handles the prefix `/`. - router.Any("/debug/fail/*any", gin.WrapH(http.StripPrefix("/debug/fail", &failpoint.HttpHandler{}))) - } - - // Promtheus metrics API - prometheus.DefaultGatherer = registry - router.Any("/metrics", gin.WrapH(promhttp.Handler())) -} diff --git a/cdc/api/router_test.go b/cdc/api/router_test.go deleted file mode 100644 index 76abe25f46b..00000000000 --- a/cdc/api/router_test.go +++ /dev/null @@ -1,60 +0,0 @@ -// 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 api - -import ( - "fmt" - "net/http" - "net/http/httptest" - "testing" - - "github.com/gin-gonic/gin" - "github.com/stretchr/testify/require" -) - -func TestPProfPath(t *testing.T) { - t.Parallel() - - router := gin.New() - RegisterRoutes(router, nil, nil) - - apis := []*testCase{ - {"/debug/pprof/", http.MethodGet}, - {"/debug/pprof/cmdline", http.MethodGet}, - {"/debug/pprof/symbol", http.MethodGet}, - // these two apis make will make ut slow - //{"/debug/pprof/profile", http.MethodGet}, - //{"/debug/pprof/trace", http.MethodGet}, - {"/debug/pprof/threadcreate", http.MethodGet}, - {"/debug/pprof/allocs", http.MethodGet}, - {"/debug/pprof/block", http.MethodGet}, - {"/debug/pprof/goroutine?debug=1", http.MethodGet}, - {"/debug/pprof/mutex?debug=1", http.MethodGet}, - } - for _, api := range apis { - w := httptest.NewRecorder() - req, _ := http.NewRequest(api.method, api.url, nil) - router.ServeHTTP(w, req) - require.Equal(t, 200, w.Code, api.String()) - } -} - -type testCase struct { - url string - method string -} - -func (a *testCase) String() string { - return fmt.Sprintf("%s:%s", a.method, a.url) -} diff --git a/cdc/api/util_test.go b/cdc/api/util_test.go index 75a2adbffe6..f9b8f36d5d5 100644 --- a/cdc/api/util_test.go +++ b/cdc/api/util_test.go @@ -22,6 +22,7 @@ import ( ) func TestIsHTTPBadRequestError(t *testing.T) { + t.Parallel() err := cerror.ErrAPIInvalidParam.GenWithStack("aa") require.Equal(t, true, IsHTTPBadRequestError(err)) err = cerror.ErrAPIInvalidParam.Wrap(errors.New("aa")) diff --git a/cdc/api/validator_test.go b/cdc/api/validator_test.go index f7d0dea59b1..967e7b11cdb 100644 --- a/cdc/api/validator_test.go +++ b/cdc/api/validator_test.go @@ -23,6 +23,7 @@ import ( ) func TestVerifyUpdateChangefeedConfig(t *testing.T) { + t.Parallel() ctx := context.Background() ctx, cancel := context.WithCancel(ctx) defer cancel() diff --git a/cdc/capture/capture.go b/cdc/capture/capture.go index 97b17835f2d..b2f8fae43aa 100644 --- a/cdc/capture/capture.go +++ b/cdc/capture/capture.go @@ -107,10 +107,14 @@ func NewCapture(pdClient pd.Client, kvStorage tidbkv.Storage, etcdClient *etcd.C } } -func NewCapture4Test() *Capture { - return &Capture{ +func NewCapture4Test(isOwner bool) *Capture { + res := &Capture{ info: &model.CaptureInfo{ID: "capture-for-test", AdvertiseAddr: "127.0.0.1", Version: "test"}, } + if isOwner { + res.owner = &owner.Owner{} + } + return res } func (c *Capture) reset(ctx context.Context) error { diff --git a/cdc/entry/mounter.go b/cdc/entry/mounter.go index e90e209e97a..5418d4f8d01 100644 --- a/cdc/entry/mounter.go +++ b/cdc/entry/mounter.go @@ -19,7 +19,7 @@ import ( "encoding/json" "fmt" "math" - "math/rand" + "sync/atomic" "time" "unsafe" @@ -39,7 +39,9 @@ import ( ) const ( - defaultOutputChanSize = 128000 + // The buffer size of input channel of each mounter worker. + // 16 is large enough, because a channel exclusively belongs to a worker. + defaultInputChanSize = 16 ) type baseKVEntry struct { @@ -67,7 +69,11 @@ type rowKVEntry struct { // Mounter is used to parse SQL events from KV events type Mounter interface { Run(ctx context.Context) error - Input() chan<- *model.PolymorphicEvent + // AddEntry accepts `model.PolymorphicEvent` with `RawKVEntry` filled and + // decodes `RawKVEntry` into `RowChangedEvent`. + // It also close `model.PolymorphicEvent.finished` channel to notify callers + // that decoding is done. + AddEntry(ctx context.Context, event *model.PolymorphicEvent) error } type mounterImpl struct { @@ -76,6 +82,9 @@ type mounterImpl struct { tz *time.Location workerNum int enableOldValue bool + + // index is an atomic variable to dispatch input events to workers. + index int64 } // NewMounter creates a mounter @@ -85,7 +94,7 @@ func NewMounter(schemaStorage SchemaStorage, workerNum int, enableOldValue bool) } chs := make([]chan *model.PolymorphicEvent, workerNum) for i := 0; i < workerNum; i++ { - chs[i] = make(chan *model.PolymorphicEvent, defaultOutputChanSize) + chs[i] = make(chan *model.PolymorphicEvent, defaultInputChanSize) } return &mounterImpl{ schemaStorage: schemaStorage, @@ -100,17 +109,34 @@ const defaultMounterWorkerNum = 32 func (m *mounterImpl) Run(ctx context.Context) error { m.tz = util.TimezoneFromCtx(ctx) errg, ctx := errgroup.WithContext(ctx) - errg.Go(func() error { - m.collectMetrics(ctx) - return nil - }) for i := 0; i < m.workerNum; i++ { index := i errg.Go(func() error { return m.codecWorker(ctx, index) }) } - return errg.Wait() + + captureAddr := util.CaptureAddrFromCtx(ctx) + changefeedID := util.ChangefeedIDFromCtx(ctx) + metricMounterInputChanSize := mounterInputChanSizeGauge.WithLabelValues(captureAddr, changefeedID) + + flushMetricsInterval := 15 * time.Second + timer := time.NewTimer(flushMetricsInterval) + defer timer.Stop() + for { + select { + // ctx.Done returns when parent ctx done or error occurs in errg. + case <-ctx.Done(): + return errg.Wait() + case <-timer.C: + chSize := 0 + for _, ch := range m.rawRowChangedChs { + chSize += len(ch) + } + metricMounterInputChanSize.Set(float64(chSize)) + timer.Reset(flushMetricsInterval) + } + } } func (m *mounterImpl) codecWorker(ctx context.Context, index int) error { @@ -148,26 +174,13 @@ func (m *mounterImpl) codecWorker(ctx context.Context, index int) error { } } -func (m *mounterImpl) Input() chan<- *model.PolymorphicEvent { - return m.rawRowChangedChs[rand.Intn(m.workerNum)] -} - -func (m *mounterImpl) collectMetrics(ctx context.Context) { - captureAddr := util.CaptureAddrFromCtx(ctx) - changefeedID := util.ChangefeedIDFromCtx(ctx) - metricMounterInputChanSize := mounterInputChanSizeGauge.WithLabelValues(captureAddr, changefeedID) - - for { - select { - case <-ctx.Done(): - return - case <-time.After(time.Second * 15): - chSize := 0 - for _, ch := range m.rawRowChangedChs { - chSize += len(ch) - } - metricMounterInputChanSize.Set(float64(chSize)) - } +func (m *mounterImpl) AddEntry(ctx context.Context, event *model.PolymorphicEvent) error { + index := atomic.AddInt64(&m.index, 1) % int64(m.workerNum) + select { + case <-ctx.Done(): + return ctx.Err() + case m.rawRowChangedChs[index] <- event: + return nil } } diff --git a/cdc/entry/schema_storage_test.go b/cdc/entry/schema_storage_test.go index 71a8abe5dd7..83b096ab66d 100644 --- a/cdc/entry/schema_storage_test.go +++ b/cdc/entry/schema_storage_test.go @@ -130,8 +130,8 @@ func TestTable(t *testing.T) { Length: 10, }, }, - Unique: true, - Primary: true, + Unique: false, + Primary: false, State: timodel.StatePublic, } // table info diff --git a/cdc/entry/schema_test_helper.go b/cdc/entry/schema_test_helper.go index cf58c893719..ca07814a7a0 100644 --- a/cdc/entry/schema_test_helper.go +++ b/cdc/entry/schema_test_helper.go @@ -14,7 +14,8 @@ package entry import ( - "github.com/pingcap/check" + "testing" + ticonfig "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" @@ -22,33 +23,34 @@ import ( timodel "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" ) // SchemaTestHelper is a test helper for schema which creates an internal tidb instance to generate DDL jobs with meta information type SchemaTestHelper struct { - c *check.C + t *testing.T tk *testkit.TestKit storage kv.Storage domain *domain.Domain } // NewSchemaTestHelper creates a SchemaTestHelper -func NewSchemaTestHelper(c *check.C) *SchemaTestHelper { +func NewSchemaTestHelper(t *testing.T) *SchemaTestHelper { store, err := mockstore.NewMockStore() - c.Assert(err, check.IsNil) + require.Nil(t, err) ticonfig.UpdateGlobal(func(conf *ticonfig.Config) { conf.AlterPrimaryKey = true }) session.SetSchemaLease(0) session.DisableStats4Test() domain, err := session.BootstrapSession(store) - c.Assert(err, check.IsNil) + require.Nil(t, err) domain.SetStatsUpdating(true) - tk := testkit.NewTestKit(c, store) + tk := testkit.NewTestKit(t, store) return &SchemaTestHelper{ - c: c, + t: t, tk: tk, storage: store, domain: domain, @@ -59,8 +61,8 @@ func NewSchemaTestHelper(c *check.C) *SchemaTestHelper { func (s *SchemaTestHelper) DDL2Job(ddl string) *timodel.Job { s.tk.MustExec(ddl) jobs, err := s.GetCurrentMeta().GetLastNHistoryDDLJobs(1) - s.c.Assert(err, check.IsNil) - s.c.Assert(jobs, check.HasLen, 1) + require.Nil(s.t, err) + require.Len(s.t, jobs, 1) return jobs[0] } @@ -72,7 +74,7 @@ func (s *SchemaTestHelper) Storage() kv.Storage { // GetCurrentMeta return the current meta snapshot func (s *SchemaTestHelper) GetCurrentMeta() *timeta.Meta { ver, err := s.storage.CurrentVersion(oracle.GlobalTxnScope) - s.c.Assert(err, check.IsNil) + require.Nil(s.t, err) return timeta.NewSnapshotMeta(s.storage.GetSnapshot(ver)) } diff --git a/cdc/http.go b/cdc/http.go index df1faa2411d..acdda995acf 100644 --- a/cdc/http.go +++ b/cdc/http.go @@ -14,90 +14,61 @@ package cdc import ( - "context" "net/http" - "strings" - "time" + "net/http/pprof" "github.com/gin-gonic/gin" - "github.com/pingcap/log" + "github.com/pingcap/failpoint" "github.com/pingcap/tiflow/cdc/api" - "github.com/pingcap/tiflow/cdc/model" - "go.uber.org/zap" -) + "github.com/pingcap/tiflow/cdc/capture" + "github.com/pingcap/tiflow/pkg/util" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promhttp" + swaggerFiles "github.com/swaggo/files" + ginSwagger "github.com/swaggo/gin-swagger" -// timeoutMiddleware wraps the request context with a timeout -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) + // use for OpenAPI online docs + _ "github.com/pingcap/tiflow/docs/swagger" +) - defer func() { - // check if context timeout was reached - if ctx.Err() == context.DeadlineExceeded { +// RegisterRoutes create a router for OpenAPI +func RegisterRoutes( + router *gin.Engine, + capture *capture.Capture, + registry prometheus.Gatherer, +) { + // online docs + router.GET("/swagger/*any", ginSwagger.WrapHandler(swaggerFiles.Handler)) - // write response and abort the request - c.Writer.WriteHeader(http.StatusGatewayTimeout) - c.Abort() - } + // Open API + api.RegisterOpenAPIRoutes(router, api.NewOpenAPI(capture)) - // cancel to clear resources after finished - cancel() - }() + // Owner API + api.RegisterOwnerAPIRoutes(router, capture) - // replace request with context wrapped request - c.Request = c.Request.WithContext(ctx) - c.Next() - } -} + // Status API + api.RegisterStatusAPIRoutes(router, capture) -func logMiddleware() gin.HandlerFunc { - return func(c *gin.Context) { - start := time.Now() - path := c.Request.URL.Path - query := c.Request.URL.RawQuery - c.Next() + // Log API + router.POST("/admin/log", gin.WrapF(api.HandleAdminLogLevel)) - cost := time.Since(start) + // pprof debug API + pprofGroup := router.Group("/debug/pprof/") + pprofGroup.GET("", gin.WrapF(pprof.Index)) + pprofGroup.GET("/:any", gin.WrapF(pprof.Index)) + pprofGroup.GET("/cmdline", gin.WrapF(pprof.Cmdline)) + pprofGroup.GET("/profile", gin.WrapF(pprof.Profile)) + pprofGroup.GET("/symbol", gin.WrapF(pprof.Symbol)) + pprofGroup.GET("/trace", gin.WrapF(pprof.Trace)) + pprofGroup.GET("/threadcreate", gin.WrapF(pprof.Handler("threadcreate").ServeHTTP)) - err := c.Errors.Last() - var stdErr error - if err != nil { - stdErr = err.Err - } - // Do not log metrics related requests when there is no error - if strings.Contains(path, "/metrics") && err == nil { - return - } - 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("duration", cost), - ) + // Failpoint API + if util.FailpointBuild { + // `http.StripPrefix` is needed because `failpoint.HttpHandler` assumes that it handles the prefix `/`. + router.Any("/debug/fail/*any", gin.WrapH(http.StripPrefix("/debug/fail", &failpoint.HttpHandler{}))) } -} -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 api.IsHTTPBadRequestError(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - } else { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) - } - c.Abort() - return - } - } + // Promtheus metrics API + prometheus.DefaultGatherer = registry + router.Any("/metrics", gin.WrapH(promhttp.Handler())) } diff --git a/cdc/http_test.go b/cdc/http_test.go index a59719e468f..39398a82639 100644 --- a/cdc/http_test.go +++ b/cdc/http_test.go @@ -1,4 +1,4 @@ -// Copyright 2020 PingCAP, Inc. +// 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. @@ -14,182 +14,80 @@ package cdc import ( - "context" - "crypto/tls" - "encoding/json" + "bytes" "fmt" "net/http" - "strings" - "sync" - "time" + "net/http/httptest" + "testing" - "github.com/pingcap/check" - "github.com/pingcap/tidb/br/pkg/httputil" + "github.com/pingcap/failpoint" + + "github.com/gin-gonic/gin" "github.com/pingcap/tiflow/cdc/capture" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/config" - cerrors "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/retry" - security2 "github.com/pingcap/tiflow/pkg/security" - "github.com/pingcap/tiflow/pkg/util/testleak" - "github.com/tikv/pd/pkg/tempurl" + "github.com/stretchr/testify/require" ) -type httpStatusSuite struct{} - -var _ = check.Suite(&httpStatusSuite{}) - -const retryTime = 20 - -func (s *httpStatusSuite) TestServerTLSWithoutCommonName(c *check.C) { - defer testleak.AfterTest(c) - addr := tempurl.Alloc()[len("http://"):] - // Do not specify common name - security, err := security2.NewCredential4Test("") - c.Assert(err, check.IsNil) - conf := config.GetDefaultServerConfig() - conf.Addr = addr - conf.AdvertiseAddr = addr - conf.Security = &security - config.StoreGlobalServerConfig(conf) - - server, err := NewServer([]string{"https://127.0.0.1:2379"}) - server.capture = capture.NewCapture4Test() - c.Assert(err, check.IsNil) - err = server.startStatusHTTP(server.tcpServer.HTTP1Listener()) - c.Assert(err, check.IsNil) - defer func() { - c.Assert(server.statusServer.Close(), check.IsNil) - }() - - statusURL := fmt.Sprintf("https://%s/api/v1/status", addr) - ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) - defer cancel() - - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - err := server.tcpServer.Run(ctx) - c.Check(err, check.ErrorMatches, ".*ErrTCPServerClosed.*") - }() - - // test cli sends request without a cert will success - err = retry.Do(ctx, func() error { - tr := &http.Transport{ - TLSClientConfig: &tls.Config{InsecureSkipVerify: true}, - } - cli := &http.Client{Transport: tr} - resp, err := cli.Get(statusURL) - if err != nil { - return err - } - decoder := json.NewDecoder(resp.Body) - captureInfo := &model.CaptureInfo{} - err = decoder.Decode(captureInfo) - c.Assert(err, check.IsNil) - c.Assert(captureInfo.ID, check.Equals, server.capture.Info().ID) - resp.Body.Close() - return nil - }, retry.WithMaxTries(retryTime), retry.WithBackoffBaseDelay(50), retry.WithIsRetryableErr(cerrors.IsRetryableError)) - c.Assert(err, check.IsNil) - - // test cli sends request with a cert will success - err = retry.Do(ctx, func() error { - tlsConfig, err := security.ToTLSConfigWithVerify() - if err != nil { - c.Assert(err, check.IsNil) - } - cli := httputil.NewClient(tlsConfig) - resp, err := cli.Get(statusURL) - if err != nil { - return err - } - decoder := json.NewDecoder(resp.Body) - captureInfo := &model.CaptureInfo{} - err = decoder.Decode(captureInfo) - c.Assert(err, check.IsNil) - c.Assert(captureInfo.ID, check.Equals, server.capture.Info().ID) - resp.Body.Close() - return nil - }, retry.WithMaxTries(retryTime), retry.WithBackoffBaseDelay(50), retry.WithIsRetryableErr(cerrors.IsRetryableError)) - c.Assert(err, check.IsNil) - - cancel() - wg.Wait() +type testCase struct { + url string + method string } -func (s *httpStatusSuite) TestServerTLSWithCommonName(c *check.C) { - defer testleak.AfterTest(c) - addr := tempurl.Alloc()[len("http://"):] - // specify a common name - security, err := security2.NewCredential4Test("test") - c.Assert(err, check.IsNil) - conf := config.GetDefaultServerConfig() - conf.Addr = addr - conf.AdvertiseAddr = addr - conf.Security = &security - config.StoreGlobalServerConfig(conf) - - server, err := NewServer([]string{"https://127.0.0.1:2379"}) - server.capture = capture.NewCapture4Test() - c.Assert(err, check.IsNil) - err = server.startStatusHTTP(server.tcpServer.HTTP1Listener()) - c.Assert(err, check.IsNil) - defer func() { - c.Assert(server.statusServer.Close(), check.IsNil) - }() - - statusURL := fmt.Sprintf("https://%s/api/v1/status", addr) - ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) - defer cancel() - - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - err := server.tcpServer.Run(ctx) - c.Check(err, check.ErrorMatches, ".*ErrTCPServerClosed.*") - }() +func (a *testCase) String() string { + return fmt.Sprintf("%s:%s", a.method, a.url) +} - // test cli sends request without a cert will fail - err = retry.Do(ctx, func() error { - tr := &http.Transport{ - TLSClientConfig: &tls.Config{InsecureSkipVerify: true}, - } - cli := &http.Client{Transport: tr} - resp, err := cli.Get(statusURL) - if err != nil { - return err - } - decoder := json.NewDecoder(resp.Body) - captureInfo := &model.CaptureInfo{} - err = decoder.Decode(captureInfo) - c.Assert(err, check.IsNil) - c.Assert(captureInfo.ID, check.Equals, server.capture.Info().ID) - resp.Body.Close() - return nil - }, retry.WithMaxTries(retryTime), retry.WithBackoffBaseDelay(50), retry.WithIsRetryableErr(cerrors.IsRetryableError)) - c.Assert(strings.Contains(err.Error(), "remote error: tls: bad certificate"), check.IsTrue) +func TestPProfPath(t *testing.T) { + router := gin.New() + RegisterRoutes(router, capture.NewCapture4Test(false), nil) + + apis := []*testCase{ + {"/debug/pprof/", http.MethodGet}, + {"/debug/pprof/cmdline", http.MethodGet}, + {"/debug/pprof/symbol", http.MethodGet}, + // these two apis make will make ut slow + //{"/debug/pprof/profile", http.MethodGet}, + //{"/debug/pprof/trace", http.MethodGet}, + {"/debug/pprof/threadcreate", http.MethodGet}, + {"/debug/pprof/allocs", http.MethodGet}, + {"/debug/pprof/block", http.MethodGet}, + {"/debug/pprof/goroutine?debug=1", http.MethodGet}, + {"/debug/pprof/mutex?debug=1", http.MethodGet}, + } + for _, api := range apis { + w := httptest.NewRecorder() + req, _ := http.NewRequest(api.method, api.url, nil) + router.ServeHTTP(w, req) + require.Equal(t, 200, w.Code, api.String()) + } +} - // test cli sends request with a cert will success - err = retry.Do(ctx, func() error { - tlsConfig, err := security.ToTLSConfigWithVerify() - if err != nil { - c.Assert(err, check.IsNil) - } - cli := httputil.NewClient(tlsConfig) - resp, err := cli.Get(statusURL) - if err != nil { - return err - } - decoder := json.NewDecoder(resp.Body) - captureInfo := &model.CaptureInfo{} - err = decoder.Decode(captureInfo) - c.Assert(err, check.IsNil) - c.Assert(captureInfo.ID, check.Equals, server.capture.Info().ID) - resp.Body.Close() - return nil - }, retry.WithMaxTries(retryTime), retry.WithBackoffBaseDelay(50), retry.WithIsRetryableErr(cerrors.IsRetryableError)) - c.Assert(err, check.IsNil) +func TestHandleFailpoint(t *testing.T) { + router := gin.New() + RegisterRoutes(router, capture.NewCapture4Test(false), nil) + fp := "github.com/pingcap/tiflow/cdc/TestHandleFailpoint" + uri := fmt.Sprintf("/debug/fail/%s", fp) + body := bytes.NewReader([]byte("return(true)")) + req, err := http.NewRequest("PUT", uri, body) + require.Nil(t, err) + w := httptest.NewRecorder() + router.ServeHTTP(w, req) + require.True(t, w.Code >= 200 && w.Code <= 300) + + failpointHit := false + failpoint.Inject("TestHandleFailpoint", func() { + failpointHit = true + }) + require.True(t, failpointHit) + + req, err = http.NewRequest("DELETE", uri, body) + require.Nil(t, err) + w = httptest.NewRecorder() + router.ServeHTTP(w, req) + require.True(t, w.Code >= 200 && w.Code <= 300) + + failpointHit = false + failpoint.Inject("TestHandleFailpoint", func() { + failpointHit = true + }) + require.False(t, failpointHit) } diff --git a/cdc/owner/barrier_test.go b/cdc/owner/barrier_test.go index 5b09cd030ba..3524b80eb48 100644 --- a/cdc/owner/barrier_test.go +++ b/cdc/owner/barrier_test.go @@ -18,50 +18,41 @@ import ( "math/rand" "testing" - "github.com/pingcap/check" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) -func Test(t *testing.T) { check.TestingT(t) } - -var _ = check.Suite(&barrierSuite{}) - -type barrierSuite struct{} - -func (s *barrierSuite) TestBarrier(c *check.C) { - defer testleak.AfterTest(c)() +func TestBarrier(t *testing.T) { b := newBarriers() b.Update(ddlJobBarrier, 2) b.Update(syncPointBarrier, 3) b.Update(finishBarrier, 1) tp, ts := b.Min() - c.Assert(tp, check.Equals, finishBarrier) - c.Assert(ts, check.Equals, uint64(1)) + require.Equal(t, tp, finishBarrier) + require.Equal(t, ts, uint64(1)) b.Update(finishBarrier, 4) tp, ts = b.Min() - c.Assert(tp, check.Equals, ddlJobBarrier) - c.Assert(ts, check.Equals, uint64(2)) + require.Equal(t, tp, ddlJobBarrier) + require.Equal(t, ts, uint64(2)) b.Remove(ddlJobBarrier) tp, ts = b.Min() - c.Assert(tp, check.Equals, syncPointBarrier) - c.Assert(ts, check.Equals, uint64(3)) + require.Equal(t, tp, syncPointBarrier) + require.Equal(t, ts, uint64(3)) b.Update(finishBarrier, 1) tp, ts = b.Min() - c.Assert(tp, check.Equals, finishBarrier) - c.Assert(ts, check.Equals, uint64(1)) + require.Equal(t, tp, finishBarrier) + require.Equal(t, ts, uint64(1)) b.Update(ddlJobBarrier, 5) tp, ts = b.Min() - c.Assert(tp, check.Equals, finishBarrier) - c.Assert(ts, check.Equals, uint64(1)) + require.Equal(t, tp, finishBarrier) + require.Equal(t, ts, uint64(1)) } -func (s *barrierSuite) TestBarrierRandom(c *check.C) { - defer testleak.AfterTest(c)() +func TestBarrierRandom(t *testing.T) { maxBarrierType := 50 maxBarrierTs := 1000000 b := newBarriers() @@ -90,7 +81,7 @@ func (s *barrierSuite) TestBarrierRandom(c *check.C) { } } tp, ts := b.Min() - c.Assert(ts, check.Equals, expectedMinTs) - c.Assert(expectedBarriers[tp], check.Equals, expectedMinTs) + require.Equal(t, ts, expectedMinTs) + require.Equal(t, expectedBarriers[tp], expectedMinTs) } } diff --git a/cdc/owner/changefeed_test.go b/cdc/owner/changefeed_test.go index e31908a18d5..427d1abf49e 100644 --- a/cdc/owner/changefeed_test.go +++ b/cdc/owner/changefeed_test.go @@ -15,13 +15,14 @@ package owner import ( "context" + "io/ioutil" "os" "path/filepath" "sync" "sync/atomic" + "testing" "time" - "github.com/pingcap/check" "github.com/pingcap/errors" timodel "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tiflow/cdc/entry" @@ -31,8 +32,8 @@ import ( "github.com/pingcap/tiflow/pkg/orchestrator" "github.com/pingcap/tiflow/pkg/pdtime" "github.com/pingcap/tiflow/pkg/txnutil/gc" - "github.com/pingcap/tiflow/pkg/util/testleak" "github.com/pingcap/tiflow/pkg/version" + "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" ) @@ -112,11 +113,7 @@ func (m *mockDDLSink) Barrier(ctx context.Context) error { return nil } -var _ = check.Suite(&changefeedSuite{}) - -type changefeedSuite struct{} - -func createChangefeed4Test(ctx cdcContext.Context, c *check.C) (*changefeed, *orchestrator.ChangefeedReactorState, +func createChangefeed4Test(ctx cdcContext.Context, t *testing.T) (*changefeed, *orchestrator.ChangefeedReactorState, map[model.CaptureID]*model.CaptureInfo, *orchestrator.ReactorStateTester) { ctx.GlobalVars().PDClient = &gc.MockPDClient{ UpdateServiceGCSafePointFunc: func(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { @@ -130,9 +127,9 @@ func createChangefeed4Test(ctx cdcContext.Context, c *check.C) (*changefeed, *or return &mockDDLSink{} }) state := orchestrator.NewChangefeedReactorState(ctx.ChangefeedVars().ID) - tester := orchestrator.NewReactorStateTester(c, state, nil) + tester := orchestrator.NewReactorStateTester(t, state, nil) state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { - c.Assert(info, check.IsNil) + require.Nil(t, info) info = ctx.ChangefeedVars().Info return info, true, nil }) @@ -142,14 +139,13 @@ func createChangefeed4Test(ctx cdcContext.Context, c *check.C) (*changefeed, *or return cf, state, captures, tester } -func (s *changefeedSuite) TestPreCheck(c *check.C) { - defer testleak.AfterTest(c)() +func TestPreCheck(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - cf, state, captures, tester := createChangefeed4Test(ctx, c) + cf, state, captures, tester := createChangefeed4Test(ctx, t) cf.Tick(ctx, state, captures) tester.MustApplyPatches() - c.Assert(state.Status, check.NotNil) - c.Assert(state.TaskStatuses, check.HasKey, ctx.GlobalVars().CaptureInfo.ID) + require.NotNil(t, state.Status) + require.Contains(t, state.TaskStatuses, ctx.GlobalVars().CaptureInfo.ID) // test clean the meta data of offline capture offlineCaputreID := "offline-capture" @@ -166,17 +162,16 @@ func (s *changefeedSuite) TestPreCheck(c *check.C) { cf.Tick(ctx, state, captures) tester.MustApplyPatches() - c.Assert(state.Status, check.NotNil) - c.Assert(state.TaskStatuses, check.HasKey, ctx.GlobalVars().CaptureInfo.ID) - c.Assert(state.TaskStatuses, check.Not(check.HasKey), offlineCaputreID) - c.Assert(state.TaskPositions, check.Not(check.HasKey), offlineCaputreID) - c.Assert(state.Workloads, check.Not(check.HasKey), offlineCaputreID) + require.NotNil(t, state.Status) + require.Contains(t, state.TaskStatuses, ctx.GlobalVars().CaptureInfo.ID) + require.NotContains(t, state.TaskStatuses, offlineCaputreID) + require.NotContains(t, state.TaskPositions, offlineCaputreID) + require.NotContains(t, state.Workloads, offlineCaputreID) } -func (s *changefeedSuite) TestInitialize(c *check.C) { - defer testleak.AfterTest(c)() +func TestInitialize(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - cf, state, captures, tester := createChangefeed4Test(ctx, c) + cf, state, captures, tester := createChangefeed4Test(ctx, t) defer cf.Close(ctx) // pre check cf.Tick(ctx, state, captures) @@ -185,13 +180,12 @@ func (s *changefeedSuite) TestInitialize(c *check.C) { // initialize cf.Tick(ctx, state, captures) tester.MustApplyPatches() - c.Assert(state.Status.CheckpointTs, check.Equals, ctx.ChangefeedVars().Info.StartTs) + require.Equal(t, state.Status.CheckpointTs, ctx.ChangefeedVars().Info.StartTs) } -func (s *changefeedSuite) TestHandleError(c *check.C) { - defer testleak.AfterTest(c)() +func TestChangefeedHandleError(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - cf, state, captures, tester := createChangefeed4Test(ctx, c) + cf, state, captures, tester := createChangefeed4Test(ctx, t) defer cf.Close(ctx) // pre check cf.Tick(ctx, state, captures) @@ -205,14 +199,12 @@ func (s *changefeedSuite) TestHandleError(c *check.C) { // handle error cf.Tick(ctx, state, captures) tester.MustApplyPatches() - c.Assert(state.Status.CheckpointTs, check.Equals, ctx.ChangefeedVars().Info.StartTs) - c.Assert(state.Info.Error.Message, check.Equals, "fake error") + require.Equal(t, state.Status.CheckpointTs, ctx.ChangefeedVars().Info.StartTs) + require.Equal(t, state.Info.Error.Message, "fake error") } -func (s *changefeedSuite) TestExecDDL(c *check.C) { - defer testleak.AfterTest(c)() - - helper := entry.NewSchemaTestHelper(c) +func TestExecDDL(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) defer helper.Close() // Creates a table, which will be deleted at the start-ts of the changefeed. // It is expected that the changefeed DOES NOT replicate this table. @@ -237,7 +229,7 @@ func (s *changefeedSuite) TestExecDDL(c *check.C) { }, }) - cf, state, captures, tester := createChangefeed4Test(ctx, c) + cf, state, captures, tester := createChangefeed4Test(ctx, t) defer cf.Close(ctx) tickThreeTime := func() { cf.Tick(ctx, state, captures) @@ -249,13 +241,12 @@ func (s *changefeedSuite) TestExecDDL(c *check.C) { } // pre check and initialize tickThreeTime() - - c.Assert(cf.schema.AllPhysicalTables(), check.HasLen, 1) - c.Assert(state.TaskStatuses[ctx.GlobalVars().CaptureInfo.ID].Operation, check.HasLen, 0) - c.Assert(state.TaskStatuses[ctx.GlobalVars().CaptureInfo.ID].Tables, check.HasLen, 0) + require.Len(t, cf.schema.AllPhysicalTables(), 1) + require.Len(t, state.TaskStatuses[ctx.GlobalVars().CaptureInfo.ID].Operation, 0) + require.Len(t, state.TaskStatuses[ctx.GlobalVars().CaptureInfo.ID].Tables, 0) job = helper.DDL2Job("drop table test0.table0") - // ddl puller resolved ts grow uo + // ddl puller resolved ts grow up mockDDLPuller := cf.ddlPuller.(*mockDDLPuller) mockDDLPuller.resolvedTs = startTs mockDDLSink := cf.sink.(*mockDDLSink) @@ -263,15 +254,15 @@ func (s *changefeedSuite) TestExecDDL(c *check.C) { mockDDLPuller.ddlQueue = append(mockDDLPuller.ddlQueue, job) // three tick to make sure all barriers set in initialize is handled tickThreeTime() - c.Assert(state.Status.CheckpointTs, check.Equals, mockDDLPuller.resolvedTs) + require.Equal(t, state.Status.CheckpointTs, mockDDLPuller.resolvedTs) // The ephemeral table should have left no trace in the schema cache - c.Assert(cf.schema.AllPhysicalTables(), check.HasLen, 0) + require.Len(t, cf.schema.AllPhysicalTables(), 0) // executing the ddl finished mockDDLSink.ddlDone = true mockDDLPuller.resolvedTs += 1000 tickThreeTime() - c.Assert(state.Status.CheckpointTs, check.Equals, mockDDLPuller.resolvedTs) + require.Equal(t, state.Status.CheckpointTs, mockDDLPuller.resolvedTs) // handle create database job = helper.DDL2Job("create database test1") @@ -279,14 +270,14 @@ func (s *changefeedSuite) TestExecDDL(c *check.C) { job.BinlogInfo.FinishedTS = mockDDLPuller.resolvedTs mockDDLPuller.ddlQueue = append(mockDDLPuller.ddlQueue, job) tickThreeTime() - c.Assert(state.Status.CheckpointTs, check.Equals, mockDDLPuller.resolvedTs) - c.Assert(mockDDLSink.ddlExecuting.Query, check.Equals, "CREATE DATABASE `test1`") + require.Equal(t, state.Status.CheckpointTs, mockDDLPuller.resolvedTs) + require.Equal(t, mockDDLSink.ddlExecuting.Query, "CREATE DATABASE `test1`") // executing the ddl finished mockDDLSink.ddlDone = true mockDDLPuller.resolvedTs += 1000 tickThreeTime() - c.Assert(state.Status.CheckpointTs, check.Equals, mockDDLPuller.resolvedTs) + require.Equal(t, state.Status.CheckpointTs, mockDDLPuller.resolvedTs) // handle create table job = helper.DDL2Job("create table test1.test1(id int primary key)") @@ -294,22 +285,22 @@ func (s *changefeedSuite) TestExecDDL(c *check.C) { job.BinlogInfo.FinishedTS = mockDDLPuller.resolvedTs mockDDLPuller.ddlQueue = append(mockDDLPuller.ddlQueue, job) tickThreeTime() - c.Assert(state.Status.CheckpointTs, check.Equals, mockDDLPuller.resolvedTs) - c.Assert(mockDDLSink.ddlExecuting.Query, check.Equals, "CREATE TABLE `test1`.`test1` (`id` INT PRIMARY KEY)") + + require.Equal(t, state.Status.CheckpointTs, mockDDLPuller.resolvedTs) + require.Equal(t, mockDDLSink.ddlExecuting.Query, "CREATE TABLE `test1`.`test1` (`id` INT PRIMARY KEY)") // executing the ddl finished mockDDLSink.ddlDone = true mockDDLPuller.resolvedTs += 1000 tickThreeTime() - c.Assert(state.TaskStatuses[ctx.GlobalVars().CaptureInfo.ID].Tables, check.HasKey, job.TableID) + require.Contains(t, state.TaskStatuses[ctx.GlobalVars().CaptureInfo.ID].Tables, job.TableID) } -func (s *changefeedSuite) TestSyncPoint(c *check.C) { - defer testleak.AfterTest(c)() +func TestSyncPoint(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) ctx.ChangefeedVars().Info.SyncPointEnabled = true ctx.ChangefeedVars().Info.SyncPointInterval = 1 * time.Second - cf, state, captures, tester := createChangefeed4Test(ctx, c) + cf, state, captures, tester := createChangefeed4Test(ctx, t) defer cf.Close(ctx) // pre check @@ -331,16 +322,15 @@ func (s *changefeedSuite) TestSyncPoint(c *check.C) { } for i := 1; i < len(mockDDLSink.syncPointHis); i++ { // check the time interval between adjacent sync points is less or equal than one second - c.Assert(mockDDLSink.syncPointHis[i]-mockDDLSink.syncPointHis[i-1], check.LessEqual, uint64(1000<<18)) + require.LessOrEqual(t, mockDDLSink.syncPointHis[i]-mockDDLSink.syncPointHis[i-1], uint64(1000<<18)) } - c.Assert(len(mockDDLSink.syncPointHis), check.GreaterEqual, 5) + require.GreaterOrEqual(t, len(mockDDLSink.syncPointHis), 5) } -func (s *changefeedSuite) TestFinished(c *check.C) { - defer testleak.AfterTest(c)() +func TestFinished(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) ctx.ChangefeedVars().Info.TargetTs = ctx.ChangefeedVars().Info.StartTs + 1000 - cf, state, captures, tester := createChangefeed4Test(ctx, c) + cf, state, captures, tester := createChangefeed4Test(ctx, t) defer cf.Close(ctx) // pre check @@ -359,17 +349,17 @@ func (s *changefeedSuite) TestFinished(c *check.C) { tester.MustApplyPatches() } - c.Assert(state.Status.CheckpointTs, check.Equals, state.Info.TargetTs) - c.Assert(state.Info.State, check.Equals, model.StateFinished) + require.Equal(t, state.Status.CheckpointTs, state.Info.TargetTs) + require.Equal(t, state.Info.State, model.StateFinished) } -func (s *changefeedSuite) TestRemoveChangefeed(c *check.C) { - defer testleak.AfterTest(c)() - +func TestRemoveChangefeed(t *testing.T) { baseCtx, cancel := context.WithCancel(context.Background()) ctx := cdcContext.NewContext4Test(baseCtx, true) info := ctx.ChangefeedVars().Info - dir := c.MkDir() + dir, err := ioutil.TempDir("", "remove-changefeed-test") + require.NoError(t, err) + defer os.RemoveAll(dir) info.Config.Consistent = &config.ConsistentConfig{ Level: "eventual", Storage: filepath.Join("nfs://", dir), @@ -378,17 +368,17 @@ func (s *changefeedSuite) TestRemoveChangefeed(c *check.C) { ID: ctx.ChangefeedVars().ID, Info: info, }) - testChangefeedReleaseResource(c, ctx, cancel, dir, true /*expectedInitialized*/) + testChangefeedReleaseResource(t, ctx, cancel, dir, true /*expectedInitialized*/) } -func (s *changefeedSuite) TestRemovePausedChangefeed(c *check.C) { - defer testleak.AfterTest(c)() - +func TestRemovePausedChangefeed(t *testing.T) { baseCtx, cancel := context.WithCancel(context.Background()) ctx := cdcContext.NewContext4Test(baseCtx, true) info := ctx.ChangefeedVars().Info info.State = model.StateStopped - dir := c.MkDir() + dir, err := ioutil.TempDir("", "remove-paused-changefeed-test") + require.NoError(t, err) + defer os.RemoveAll(dir) info.Config.Consistent = &config.ConsistentConfig{ Level: "eventual", Storage: filepath.Join("nfs://", dir), @@ -397,17 +387,17 @@ func (s *changefeedSuite) TestRemovePausedChangefeed(c *check.C) { ID: ctx.ChangefeedVars().ID, Info: info, }) - testChangefeedReleaseResource(c, ctx, cancel, dir, false /*expectedInitialized*/) + testChangefeedReleaseResource(t, ctx, cancel, dir, false /*expectedInitialized*/) } func testChangefeedReleaseResource( - c *check.C, + t *testing.T, ctx cdcContext.Context, cancel context.CancelFunc, redoLogDir string, expectedInitialized bool, ) { - cf, state, captures, tester := createChangefeed4Test(ctx, c) + cf, state, captures, tester := createChangefeed4Test(ctx, t) // pre check cf.Tick(ctx, state, captures) @@ -416,7 +406,7 @@ func testChangefeedReleaseResource( // initialize cf.Tick(ctx, state, captures) tester.MustApplyPatches() - c.Assert(cf.initialized, check.Equals, expectedInitialized) + require.Equal(t, cf.initialized, expectedInitialized) // remove changefeed from state manager by admin job cf.feedStateManager.PushAdminJob(&model.AdminJob{ @@ -425,15 +415,14 @@ func testChangefeedReleaseResource( }) // changefeed tick will release resources err := cf.tick(ctx, state, captures) - c.Assert(err, check.IsNil) + require.Nil(t, err) cancel() // check redo log dir is deleted _, err = os.Stat(redoLogDir) - c.Assert(os.IsNotExist(err), check.IsTrue) + require.True(t, os.IsNotExist(err)) } -func (s *changefeedSuite) TestAddSpecialComment(c *check.C) { - defer testleak.AfterTest(c)() +func TestAddSpecialComment(t *testing.T) { testCase := []struct { input string result string @@ -565,10 +554,10 @@ func (s *changefeedSuite) TestAddSpecialComment(c *check.C) { } for _, ca := range testCase { re, err := addSpecialComment(ca.input) - c.Check(err, check.IsNil) - c.Check(re, check.Equals, ca.result) + require.Nil(t, err) + require.Equal(t, re, ca.result) } - c.Assert(func() { + require.Panics(t, func() { _, _ = addSpecialComment("alter table t force, auto_increment = 12;alter table t force, auto_increment = 12;") - }, check.Panics, "invalid ddlQuery statement size") + }, "invalid ddlQuery statement size") } diff --git a/cdc/owner/ddl_puller_test.go b/cdc/owner/ddl_puller_test.go index 654fa371060..08f5a7bbd26 100644 --- a/cdc/owner/ddl_puller_test.go +++ b/cdc/owner/ddl_puller_test.go @@ -18,10 +18,10 @@ import ( "encoding/json" "sync" "sync/atomic" + "testing" "time" "github.com/benbjohnson/clock" - "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/log" timodel "github.com/pingcap/tidb/parser/model" @@ -29,25 +29,21 @@ import ( "github.com/pingcap/tiflow/cdc/model" cdcContext "github.com/pingcap/tiflow/pkg/context" "github.com/pingcap/tiflow/pkg/retry" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" "go.uber.org/zap" "go.uber.org/zap/zaptest/observer" ) -var _ = check.Suite(&ddlPullerSuite{}) - -type ddlPullerSuite struct{} - type mockPuller struct { - c *check.C + t *testing.T inCh chan *model.RawKVEntry outCh chan *model.RawKVEntry resolvedTs model.Ts } -func newMockPuller(c *check.C, startTs model.Ts) *mockPuller { +func newMockPuller(t *testing.T, startTs model.Ts) *mockPuller { return &mockPuller{ - c: c, + t: t, inCh: make(chan *model.RawKVEntry), outCh: make(chan *model.RawKVEntry), resolvedTs: startTs - 1, @@ -84,7 +80,7 @@ func (m *mockPuller) append(e *model.RawKVEntry) { func (m *mockPuller) appendDDL(job *timodel.Job) { b, err := json.Marshal(job) - m.c.Assert(err, check.IsNil) + require.Nil(m.t, err) ek := []byte("m") ek = codec.EncodeBytes(ek, []byte("DDLJobList")) ek = codec.EncodeUint(ek, uint64('l')) @@ -106,13 +102,12 @@ func (m *mockPuller) appendResolvedTs(ts model.Ts) { }) } -func (s *ddlPullerSuite) TestPuller(c *check.C) { - defer testleak.AfterTest(c)() +func TestPuller(t *testing.T) { startTs := uint64(10) - mockPuller := newMockPuller(c, startTs) + mockPuller := newMockPuller(t, startTs) ctx := cdcContext.NewBackendContext4Test(true) p, err := newDDLPuller(ctx, startTs) - c.Assert(err, check.IsNil) + require.Nil(t, err) p.(*ddlPullerImpl).puller = mockPuller var wg sync.WaitGroup wg.Add(1) @@ -122,22 +117,22 @@ func (s *ddlPullerSuite) TestPuller(c *check.C) { if errors.Cause(err) == context.Canceled { err = nil } - c.Assert(err, check.IsNil) + require.Nil(t, err) }() defer wg.Wait() defer p.Close() // test initialize state resolvedTs, ddl := p.FrontDDL() - c.Assert(resolvedTs, check.Equals, startTs) - c.Assert(ddl, check.IsNil) + require.Equal(t, resolvedTs, startTs) + require.Nil(t, ddl) resolvedTs, ddl = p.PopFrontDDL() - c.Assert(resolvedTs, check.Equals, startTs) - c.Assert(ddl, check.IsNil) + require.Equal(t, resolvedTs, startTs) + require.Nil(t, ddl) // test send resolvedTs mockPuller.appendResolvedTs(15) - waitResolvedTsGrowing(c, p, 15) + waitResolvedTsGrowing(t, p, 15) // test send ddl job out of order mockPuller.appendDDL(&timodel.Job{ @@ -155,23 +150,23 @@ func (s *ddlPullerSuite) TestPuller(c *check.C) { BinlogInfo: &timodel.HistoryInfo{FinishedTS: 16}, }) resolvedTs, ddl = p.FrontDDL() - c.Assert(resolvedTs, check.Equals, uint64(15)) - c.Assert(ddl, check.IsNil) + require.Equal(t, resolvedTs, uint64(15)) + require.Nil(t, ddl) mockPuller.appendResolvedTs(20) - waitResolvedTsGrowing(c, p, 16) + waitResolvedTsGrowing(t, p, 16) resolvedTs, ddl = p.FrontDDL() - c.Assert(resolvedTs, check.Equals, uint64(16)) - c.Assert(ddl.ID, check.Equals, int64(1)) + require.Equal(t, resolvedTs, uint64(16)) + require.Equal(t, ddl.ID, int64(1)) resolvedTs, ddl = p.PopFrontDDL() - c.Assert(resolvedTs, check.Equals, uint64(16)) - c.Assert(ddl.ID, check.Equals, int64(1)) + require.Equal(t, resolvedTs, uint64(16)) + require.Equal(t, ddl.ID, int64(1)) // DDL could be processed with a delay, wait here for a pending DDL job is added - waitResolvedTsGrowing(c, p, 18) + waitResolvedTsGrowing(t, p, 18) resolvedTs, ddl = p.PopFrontDDL() - c.Assert(resolvedTs, check.Equals, uint64(18)) - c.Assert(ddl.ID, check.Equals, int64(2)) + require.Equal(t, resolvedTs, uint64(18)) + require.Equal(t, ddl.ID, int64(2)) // test add ddl job repeated mockPuller.appendDDL(&timodel.Job{ @@ -189,18 +184,18 @@ func (s *ddlPullerSuite) TestPuller(c *check.C) { BinlogInfo: &timodel.HistoryInfo{FinishedTS: 25}, }) mockPuller.appendResolvedTs(30) - waitResolvedTsGrowing(c, p, 25) + waitResolvedTsGrowing(t, p, 25) resolvedTs, ddl = p.PopFrontDDL() - c.Assert(resolvedTs, check.Equals, uint64(25)) - c.Assert(ddl.ID, check.Equals, int64(3)) + require.Equal(t, resolvedTs, uint64(25)) + require.Equal(t, ddl.ID, int64(3)) _, ddl = p.PopFrontDDL() - c.Assert(ddl, check.IsNil) + require.Nil(t, ddl) - waitResolvedTsGrowing(c, p, 30) + waitResolvedTsGrowing(t, p, 30) resolvedTs, ddl = p.PopFrontDDL() - c.Assert(resolvedTs, check.Equals, uint64(30)) - c.Assert(ddl, check.IsNil) + require.Equal(t, resolvedTs, uint64(30)) + require.Nil(t, ddl) // test add invalid ddl job mockPuller.appendDDL(&timodel.Job{ @@ -218,15 +213,14 @@ func (s *ddlPullerSuite) TestPuller(c *check.C) { BinlogInfo: &timodel.HistoryInfo{FinishedTS: 36}, }) mockPuller.appendResolvedTs(40) - waitResolvedTsGrowing(c, p, 40) + waitResolvedTsGrowing(t, p, 40) resolvedTs, ddl = p.PopFrontDDL() // no ddl should be received - c.Assert(resolvedTs, check.Equals, uint64(40)) - c.Assert(ddl, check.IsNil) + require.Equal(t, resolvedTs, uint64(40)) + require.Nil(t, ddl) } -func (*ddlPullerSuite) TestResolvedTsStuck(c *check.C) { - defer testleak.AfterTest(c)() +func TestResolvedTsStuck(t *testing.T) { // For observing the logs zapcore, logs := observer.New(zap.WarnLevel) conf := &log.Config{Level: "warn", File: log.FileLogConfig{}} @@ -236,10 +230,10 @@ func (*ddlPullerSuite) TestResolvedTsStuck(c *check.C) { defer restoreFn() startTs := uint64(10) - mockPuller := newMockPuller(c, startTs) + mockPuller := newMockPuller(t, startTs) ctx := cdcContext.NewBackendContext4Test(true) p, err := newDDLPuller(ctx, startTs) - c.Assert(err, check.IsNil) + require.Nil(t, err) mockClock := clock.NewMock() p.(*ddlPullerImpl).clock = mockClock @@ -253,22 +247,22 @@ func (*ddlPullerSuite) TestResolvedTsStuck(c *check.C) { if errors.Cause(err) == context.Canceled { err = nil } - c.Assert(err, check.IsNil) + require.Nil(t, err) }() defer wg.Wait() defer p.Close() // test initialize state resolvedTs, ddl := p.FrontDDL() - c.Assert(resolvedTs, check.Equals, startTs) - c.Assert(ddl, check.IsNil) + require.Equal(t, resolvedTs, startTs) + require.Nil(t, ddl) resolvedTs, ddl = p.PopFrontDDL() - c.Assert(resolvedTs, check.Equals, startTs) - c.Assert(ddl, check.IsNil) + require.Equal(t, resolvedTs, startTs) + require.Nil(t, ddl) mockPuller.appendResolvedTs(30) - waitResolvedTsGrowing(c, p, 30) - c.Assert(logs.Len(), check.Equals, 0) + waitResolvedTsGrowing(t, p, 30) + require.Equal(t, logs.Len(), 0) mockClock.Add(2 * ownerDDLPullerStuckWarnTimeout) for i := 0; i < 20; i++ { @@ -278,17 +272,17 @@ func (*ddlPullerSuite) TestResolvedTsStuck(c *check.C) { } time.Sleep(10 * time.Millisecond) if i == 19 { - c.Fatal("warning log not printed") + t.Fatal("warning log not printed") } } mockPuller.appendResolvedTs(40) - waitResolvedTsGrowing(c, p, 40) + waitResolvedTsGrowing(t, p, 40) } // waitResolvedTsGrowing can wait the first DDL reaches targetTs or if no pending // DDL, DDL resolved ts reaches targetTs. -func waitResolvedTsGrowing(c *check.C, p DDLPuller, targetTs model.Ts) { +func waitResolvedTsGrowing(t *testing.T, p DDLPuller, targetTs model.Ts) { err := retry.Do(context.Background(), func() error { resolvedTs, _ := p.FrontDDL() if resolvedTs < targetTs { @@ -296,5 +290,5 @@ func waitResolvedTsGrowing(c *check.C, p DDLPuller, targetTs model.Ts) { } return nil }, retry.WithBackoffBaseDelay(20), retry.WithMaxTries(100)) - c.Assert(err, check.IsNil) + require.Nil(t, err) } diff --git a/cdc/owner/ddl_sink_test.go b/cdc/owner/ddl_sink_test.go index 94dff72381b..3a6d2137b66 100644 --- a/cdc/owner/ddl_sink_test.go +++ b/cdc/owner/ddl_sink_test.go @@ -17,22 +17,18 @@ import ( "context" "sync" "sync/atomic" + "testing" "time" - "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink" cdcContext "github.com/pingcap/tiflow/pkg/context" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/retry" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) -var _ = check.Suite(&ddlSinkSuite{}) - -type ddlSinkSuite struct{} - type mockSink struct { sink.Sink checkpointTs model.Ts @@ -78,8 +74,7 @@ func newDDLSink4Test() (DDLSink, *mockSink) { return ddlSink, mockSink } -func (s *ddlSinkSuite) TestCheckpoint(c *check.C) { - defer testleak.AfterTest(c)() +func TestCheckpoint(t *testing.T) { ddlSink, mSink := newDDLSink4Test() ctx := cdcContext.NewBackendContext4Test(true) ctx, cancel := cdcContext.WithCancel(ctx) @@ -98,13 +93,12 @@ func (s *ddlSinkSuite) TestCheckpoint(c *check.C) { }, retry.WithBackoffBaseDelay(100), retry.WithMaxTries(30)) } ddlSink.emitCheckpointTs(ctx, 1) - c.Assert(waitCheckpointGrowingUp(mSink, 1), check.IsNil) + require.Nil(t, waitCheckpointGrowingUp(mSink, 1)) ddlSink.emitCheckpointTs(ctx, 10) - c.Assert(waitCheckpointGrowingUp(mSink, 10), check.IsNil) + require.Nil(t, waitCheckpointGrowingUp(mSink, 10)) } -func (s *ddlSinkSuite) TestExecDDL(c *check.C) { - defer testleak.AfterTest(c)() +func TestExecDDLEvents(t *testing.T) { ddlSink, mSink := newDDLSink4Test() ctx := cdcContext.NewBackendContext4Test(true) ctx, cancel := cdcContext.WithCancel(ctx) @@ -123,17 +117,16 @@ func (s *ddlSinkSuite) TestExecDDL(c *check.C) { for _, event := range ddlEvents { for { done, err := ddlSink.emitDDLEvent(ctx, event) - c.Assert(err, check.IsNil) + require.Nil(t, err) if done { - c.Assert(mSink.GetDDL(), check.DeepEquals, event) + require.Equal(t, mSink.GetDDL(), event) break } } } } -func (s *ddlSinkSuite) TestExecDDLError(c *check.C) { - defer testleak.AfterTest(c)() +func TestExecDDLError(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) var ( @@ -165,24 +158,24 @@ func (s *ddlSinkSuite) TestExecDDLError(c *check.C) { ddl1 := &model.DDLEvent{CommitTs: 1} for { done, err := ddlSink.emitDDLEvent(ctx, ddl1) - c.Assert(err, check.IsNil) + require.Nil(t, err) if done { - c.Assert(mSink.GetDDL(), check.DeepEquals, ddl1) + require.Equal(t, mSink.GetDDL(), ddl1) break } } - c.Assert(resultErr, check.IsNil) + require.Nil(t, resultErr) mSink.ddlError = cerror.ErrExecDDLFailed.GenWithStackByArgs() ddl2 := &model.DDLEvent{CommitTs: 2} for { done, err := ddlSink.emitDDLEvent(ctx, ddl2) - c.Assert(err, check.IsNil) + require.Nil(t, err) if done || readResultErr() != nil { - c.Assert(mSink.GetDDL(), check.DeepEquals, ddl2) + require.Equal(t, mSink.GetDDL(), ddl2) break } } - c.Assert(cerror.ErrExecDDLFailed.Equal(readResultErr()), check.IsTrue) + require.True(t, cerror.ErrExecDDLFailed.Equal(readResultErr())) } diff --git a/cdc/owner/feed_state_manager_test.go b/cdc/owner/feed_state_manager_test.go index cccb24b9833..3eb931722e4 100644 --- a/cdc/owner/feed_state_manager_test.go +++ b/cdc/owner/feed_state_manager_test.go @@ -14,38 +14,33 @@ package owner import ( + "testing" "time" - "github.com/pingcap/check" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" cdcContext "github.com/pingcap/tiflow/pkg/context" "github.com/pingcap/tiflow/pkg/orchestrator" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) -var _ = check.Suite(&feedStateManagerSuite{}) - -type feedStateManagerSuite struct{} - -func (s *feedStateManagerSuite) TestHandleJob(c *check.C) { - defer testleak.AfterTest(c)() +func TestHandleJob(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) manager := newFeedStateManager4Test() state := orchestrator.NewChangefeedReactorState(ctx.ChangefeedVars().ID) - tester := orchestrator.NewReactorStateTester(c, state, nil) + tester := orchestrator.NewReactorStateTester(t, state, nil) state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { - c.Assert(info, check.IsNil) + require.Nil(t, info) return &model.ChangeFeedInfo{SinkURI: "123", Config: &config.ReplicaConfig{}}, true, nil }) state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { - c.Assert(status, check.IsNil) + require.Nil(t, status) return &model.ChangeFeedStatus{}, true, nil }) tester.MustApplyPatches() manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsTrue) + require.True(t, manager.ShouldRunning()) // an admin job which of changefeed is not match manager.PushAdminJob(&model.AdminJob{ @@ -54,7 +49,7 @@ func (s *feedStateManagerSuite) TestHandleJob(c *check.C) { }) manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsTrue) + require.True(t, manager.ShouldRunning()) // a running can not be resume manager.PushAdminJob(&model.AdminJob{ @@ -63,7 +58,7 @@ func (s *feedStateManagerSuite) TestHandleJob(c *check.C) { }) manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsTrue) + require.True(t, manager.ShouldRunning()) // stop a changefeed manager.PushAdminJob(&model.AdminJob{ @@ -72,11 +67,12 @@ func (s *feedStateManagerSuite) TestHandleJob(c *check.C) { }) manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsFalse) - c.Assert(manager.ShouldRemoved(), check.IsFalse) - c.Assert(state.Info.State, check.Equals, model.StateStopped) - c.Assert(state.Info.AdminJobType, check.Equals, model.AdminStop) - c.Assert(state.Status.AdminJobType, check.Equals, model.AdminStop) + + require.False(t, manager.ShouldRunning()) + require.False(t, manager.ShouldRemoved()) + require.Equal(t, state.Info.State, model.StateStopped) + require.Equal(t, state.Info.AdminJobType, model.AdminStop) + require.Equal(t, state.Status.AdminJobType, model.AdminStop) // resume a changefeed manager.PushAdminJob(&model.AdminJob{ @@ -85,11 +81,11 @@ func (s *feedStateManagerSuite) TestHandleJob(c *check.C) { }) manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsTrue) - c.Assert(manager.ShouldRemoved(), check.IsFalse) - c.Assert(state.Info.State, check.Equals, model.StateNormal) - c.Assert(state.Info.AdminJobType, check.Equals, model.AdminNone) - c.Assert(state.Status.AdminJobType, check.Equals, model.AdminNone) + require.True(t, manager.ShouldRunning()) + require.False(t, manager.ShouldRemoved()) + require.Equal(t, state.Info.State, model.StateNormal) + require.Equal(t, state.Info.AdminJobType, model.AdminNone) + require.Equal(t, state.Status.AdminJobType, model.AdminNone) // remove a changefeed manager.PushAdminJob(&model.AdminJob{ @@ -98,51 +94,51 @@ func (s *feedStateManagerSuite) TestHandleJob(c *check.C) { }) manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsFalse) - c.Assert(manager.ShouldRemoved(), check.IsTrue) - c.Assert(state.Exist(), check.IsFalse) + + require.False(t, manager.ShouldRunning()) + require.True(t, manager.ShouldRemoved()) + require.False(t, state.Exist()) } -func (s *feedStateManagerSuite) TestMarkFinished(c *check.C) { - defer testleak.AfterTest(c)() +func TestMarkFinished(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) manager := newFeedStateManager4Test() state := orchestrator.NewChangefeedReactorState(ctx.ChangefeedVars().ID) - tester := orchestrator.NewReactorStateTester(c, state, nil) + tester := orchestrator.NewReactorStateTester(t, state, nil) state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { - c.Assert(info, check.IsNil) + require.Nil(t, info) return &model.ChangeFeedInfo{SinkURI: "123", Config: &config.ReplicaConfig{}}, true, nil }) state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { - c.Assert(status, check.IsNil) + require.Nil(t, status) return &model.ChangeFeedStatus{}, true, nil }) tester.MustApplyPatches() manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsTrue) + require.True(t, manager.ShouldRunning()) manager.MarkFinished() manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsFalse) - c.Assert(state.Info.State, check.Equals, model.StateFinished) - c.Assert(state.Info.AdminJobType, check.Equals, model.AdminFinish) - c.Assert(state.Status.AdminJobType, check.Equals, model.AdminFinish) + + require.False(t, manager.ShouldRunning()) + require.Equal(t, state.Info.State, model.StateFinished) + require.Equal(t, state.Info.AdminJobType, model.AdminFinish) + require.Equal(t, state.Status.AdminJobType, model.AdminFinish) } -func (s *feedStateManagerSuite) TestCleanUpInfos(c *check.C) { - defer testleak.AfterTest(c)() +func TestCleanUpInfos(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) manager := newFeedStateManager4Test() state := orchestrator.NewChangefeedReactorState(ctx.ChangefeedVars().ID) - tester := orchestrator.NewReactorStateTester(c, state, nil) + tester := orchestrator.NewReactorStateTester(t, state, nil) state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { - c.Assert(info, check.IsNil) + require.Nil(t, info) return &model.ChangeFeedInfo{SinkURI: "123", Config: &config.ReplicaConfig{}}, true, nil }) state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { - c.Assert(status, check.IsNil) + require.Nil(t, status) return &model.ChangeFeedStatus{}, true, nil }) state.PatchTaskStatus(ctx.GlobalVars().CaptureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { @@ -155,37 +151,36 @@ func (s *feedStateManagerSuite) TestCleanUpInfos(c *check.C) { return model.TaskWorkload{}, true, nil }) tester.MustApplyPatches() - c.Assert(state.TaskStatuses, check.HasKey, ctx.GlobalVars().CaptureInfo.ID) - c.Assert(state.TaskPositions, check.HasKey, ctx.GlobalVars().CaptureInfo.ID) - c.Assert(state.Workloads, check.HasKey, ctx.GlobalVars().CaptureInfo.ID) + require.Contains(t, state.TaskStatuses, ctx.GlobalVars().CaptureInfo.ID) + require.Contains(t, state.TaskPositions, ctx.GlobalVars().CaptureInfo.ID) + require.Contains(t, state.Workloads, ctx.GlobalVars().CaptureInfo.ID) manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsTrue) + require.True(t, manager.ShouldRunning()) manager.MarkFinished() manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsFalse) - c.Assert(state.Info.State, check.Equals, model.StateFinished) - c.Assert(state.Info.AdminJobType, check.Equals, model.AdminFinish) - c.Assert(state.Status.AdminJobType, check.Equals, model.AdminFinish) - c.Assert(state.TaskStatuses, check.Not(check.HasKey), ctx.GlobalVars().CaptureInfo.ID) - c.Assert(state.TaskPositions, check.Not(check.HasKey), ctx.GlobalVars().CaptureInfo.ID) - c.Assert(state.Workloads, check.Not(check.HasKey), ctx.GlobalVars().CaptureInfo.ID) + require.False(t, manager.ShouldRunning()) + require.Equal(t, state.Info.State, model.StateFinished) + require.Equal(t, state.Info.AdminJobType, model.AdminFinish) + require.Equal(t, state.Status.AdminJobType, model.AdminFinish) + require.NotContains(t, state.TaskStatuses, ctx.GlobalVars().CaptureInfo.ID) + require.NotContains(t, state.TaskPositions, ctx.GlobalVars().CaptureInfo.ID) + require.NotContains(t, state.Workloads, ctx.GlobalVars().CaptureInfo.ID) } -func (s *feedStateManagerSuite) TestHandleError(c *check.C) { - defer testleak.AfterTest(c)() +func TestHandleError(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) manager := newFeedStateManager4Test() state := orchestrator.NewChangefeedReactorState(ctx.ChangefeedVars().ID) - tester := orchestrator.NewReactorStateTester(c, state, nil) + tester := orchestrator.NewReactorStateTester(t, state, nil) state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { - c.Assert(info, check.IsNil) + require.Nil(t, info) return &model.ChangeFeedInfo{SinkURI: "123", Config: &config.ReplicaConfig{}}, true, nil }) state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { - c.Assert(status, check.IsNil) + require.Nil(t, status) return &model.ChangeFeedStatus{}, true, nil }) state.PatchTaskStatus(ctx.GlobalVars().CaptureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { @@ -206,7 +201,7 @@ func (s *feedStateManagerSuite) TestHandleError(c *check.C) { } for _, d := range intervals { - c.Assert(manager.ShouldRunning(), check.IsTrue) + require.True(t, manager.ShouldRunning()) state.PatchTaskPosition(ctx.GlobalVars().CaptureInfo.ID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { return &model.TaskPosition{Error: &model.RunningError{ Addr: ctx.GlobalVars().CaptureInfo.AdvertiseAddr, @@ -217,17 +212,17 @@ func (s *feedStateManagerSuite) TestHandleError(c *check.C) { tester.MustApplyPatches() manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsFalse) + require.False(t, manager.ShouldRunning()) time.Sleep(d) manager.Tick(state) tester.MustApplyPatches() } - c.Assert(manager.ShouldRunning(), check.IsFalse) - c.Assert(manager.ShouldRemoved(), check.IsFalse) - c.Assert(state.Info.State, check.Equals, model.StateFailed) - c.Assert(state.Info.AdminJobType, check.Equals, model.AdminStop) - c.Assert(state.Status.AdminJobType, check.Equals, model.AdminStop) + require.False(t, manager.ShouldRunning()) + require.False(t, manager.ShouldRemoved()) + require.Equal(t, state.Info.State, model.StateFailed) + require.Equal(t, state.Info.AdminJobType, model.AdminStop) + require.Equal(t, state.Status.AdminJobType, model.AdminStop) // admin resume must retry changefeed immediately. manager.PushAdminJob(&model.AdminJob{ @@ -237,26 +232,56 @@ func (s *feedStateManagerSuite) TestHandleError(c *check.C) { }) manager.Tick(state) tester.MustApplyPatches() - c.Assert(manager.ShouldRunning(), check.IsTrue) - c.Assert(manager.ShouldRemoved(), check.IsFalse) - c.Assert(state.Info.State, check.Equals, model.StateNormal) - c.Assert(state.Info.AdminJobType, check.Equals, model.AdminNone) - c.Assert(state.Status.AdminJobType, check.Equals, model.AdminNone) + require.True(t, manager.ShouldRunning()) + require.False(t, manager.ShouldRemoved()) + require.Equal(t, state.Info.State, model.StateNormal) + require.Equal(t, state.Info.AdminJobType, model.AdminNone) + require.Equal(t, state.Status.AdminJobType, model.AdminNone) +} + +func TestHandleFastFailError(t *testing.T) { + ctx := cdcContext.NewBackendContext4Test(true) + manager := new(feedStateManager) + state := orchestrator.NewChangefeedReactorState(ctx.ChangefeedVars().ID) + tester := orchestrator.NewReactorStateTester(t, state, nil) + state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { + require.Nil(t, info) + return &model.ChangeFeedInfo{SinkURI: "123", Config: &config.ReplicaConfig{}}, true, nil + }) + state.PatchTaskPosition(ctx.GlobalVars().CaptureInfo.ID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { + return &model.TaskPosition{Error: &model.RunningError{ + Addr: ctx.GlobalVars().CaptureInfo.AdvertiseAddr, + Code: "CDC:ErrGCTTLExceeded", + Message: "fake error for test", + }}, true, nil + }) + tester.MustApplyPatches() + manager.Tick(state) + // test handling fast failed error with non-nil ChangeFeedInfo + tester.MustApplyPatches() + // test handling fast failed error with nil ChangeFeedInfo + // set info to nil when this patch is applied + state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { + return nil, true, nil + }) + manager.Tick(state) + // When the patches are applied, the callback function of PatchInfo in feedStateManager.HandleError will be called. + // At that time, the nil pointer will be checked instead of throwing a panic. See issue #3128 for more detail. + tester.MustApplyPatches() } -func (s *feedStateManagerSuite) TestChangefeedStatusNotExist(c *check.C) { - defer testleak.AfterTest(c)() +func TestChangefeedStatusNotExist(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) manager := newFeedStateManager4Test() state := orchestrator.NewChangefeedReactorState(ctx.ChangefeedVars().ID) - tester := orchestrator.NewReactorStateTester(c, state, map[string]string{ + tester := orchestrator.NewReactorStateTester(t, state, map[string]string{ "/tidb/cdc/capture/d563bfc0-f406-4f34-bc7d-6dc2e35a44e5": `{"id":"d563bfc0-f406-4f34-bc7d-6dc2e35a44e5","address":"172.16.6.147:8300","version":"v5.0.0-master-dirty"}`, "/tidb/cdc/changefeed/info/" + ctx.ChangefeedVars().ID: `{"sink-uri":"blackhole:///","opts":{},"create-time":"2021-06-05T00:44:15.065939487+08:00","start-ts":425381670108266496,"target-ts":0,"admin-job-type":1,"sort-engine":"unified","config":{"case-sensitive":true,"enable-old-value":true,"force-replicate":false,"check-gc-safe-point":true,"filter":{"rules":["*.*"],"ignore-txn-start-ts":null},"mounter":{"worker-num":16},"sink":{"dispatchers":null,"protocol":"open-protocol"},"cyclic-replication":{"enable":false,"replica-id":0,"filter-replica-ids":null,"id-buckets":0,"sync-ddl":false},"scheduler":{"type":"table-number","polling-time":-1}},"state":"failed","history":[],"error":{"addr":"172.16.6.147:8300","code":"CDC:ErrSnapshotLostByGC","message":"[CDC:ErrSnapshotLostByGC]fail to create or maintain changefeed due to snapshot loss caused by GC. checkpoint-ts 425381670108266496 is earlier than GC safepoint at 0"},"sync-point-enabled":false,"sync-point-interval":600000000000,"creator-version":"v5.0.0-master-dirty"}`, "/tidb/cdc/owner/156579d017f84a68": "d563bfc0-f406-4f34-bc7d-6dc2e35a44e5", }) manager.Tick(state) - c.Assert(manager.ShouldRunning(), check.IsFalse) - c.Assert(manager.ShouldRemoved(), check.IsFalse) + require.False(t, manager.ShouldRunning()) + require.False(t, manager.ShouldRemoved()) tester.MustApplyPatches() manager.PushAdminJob(&model.AdminJob{ @@ -265,9 +290,9 @@ func (s *feedStateManagerSuite) TestChangefeedStatusNotExist(c *check.C) { Opts: &model.AdminJobOption{ForceRemove: true}, }) manager.Tick(state) - c.Assert(manager.ShouldRunning(), check.IsFalse) - c.Assert(manager.ShouldRemoved(), check.IsTrue) + require.False(t, manager.ShouldRunning()) + require.True(t, manager.ShouldRemoved()) tester.MustApplyPatches() - c.Assert(state.Info, check.IsNil) - c.Assert(state.Exist(), check.IsFalse) + require.Nil(t, state.Info) + require.False(t, state.Exist()) } diff --git a/cdc/cdc_test.go b/cdc/owner/main_test.go similarity index 77% rename from cdc/cdc_test.go rename to cdc/owner/main_test.go index 9dd368511d5..2019d2834ec 100644 --- a/cdc/cdc_test.go +++ b/cdc/owner/main_test.go @@ -1,4 +1,4 @@ -// Copyright 2020 PingCAP, Inc. +// Copyright 2022 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -11,12 +11,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -package cdc +package owner import ( "testing" - "github.com/pingcap/check" + "github.com/pingcap/tiflow/pkg/leakutil" ) -func TestSuite(t *testing.T) { check.TestingT(t) } +func TestMain(m *testing.M) { + leakutil.SetUpLeakTest(m) +} diff --git a/cdc/owner/owner_test.go b/cdc/owner/owner_test.go index b836d174e32..bcca13f5dad 100644 --- a/cdc/owner/owner_test.go +++ b/cdc/owner/owner_test.go @@ -18,9 +18,9 @@ import ( "context" "fmt" "math" + "testing" "time" - "github.com/pingcap/check" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" cdcContext "github.com/pingcap/tiflow/pkg/context" @@ -28,14 +28,10 @@ import ( "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/orchestrator" "github.com/pingcap/tiflow/pkg/txnutil/gc" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" ) -var _ = check.Suite(&ownerSuite{}) - -type ownerSuite struct{} - type mockManager struct { gc.Manager } @@ -48,7 +44,7 @@ func (m *mockManager) CheckStaleCheckpointTs( var _ gc.Manager = (*mockManager)(nil) -func createOwner4Test(ctx cdcContext.Context, c *check.C) (*Owner, *orchestrator.GlobalReactorState, *orchestrator.ReactorStateTester) { +func createOwner4Test(ctx cdcContext.Context, t *testing.T) (*Owner, *orchestrator.GlobalReactorState, *orchestrator.ReactorStateTester) { ctx.GlobalVars().PDClient = &gc.MockPDClient{ UpdateServiceGCSafePointFunc: func(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { return safePoint, nil @@ -62,7 +58,7 @@ func createOwner4Test(ctx cdcContext.Context, c *check.C) (*Owner, *orchestrator ctx.GlobalVars().PDClient, ) state := orchestrator.NewGlobalState() - tester := orchestrator.NewReactorStateTester(c, state, nil) + tester := orchestrator.NewReactorStateTester(t, state, nil) // set captures cdcKey := etcd.CDCKey{ @@ -70,18 +66,17 @@ func createOwner4Test(ctx cdcContext.Context, c *check.C) (*Owner, *orchestrator CaptureID: ctx.GlobalVars().CaptureInfo.ID, } captureBytes, err := ctx.GlobalVars().CaptureInfo.Marshal() - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustUpdate(cdcKey.String(), captureBytes) return owner, state, tester } -func (s *ownerSuite) TestCreateRemoveChangefeed(c *check.C) { - defer testleak.AfterTest(c)() +func TestCreateRemoveChangefeed(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(false) ctx, cancel := cdcContext.WithCancel(ctx) defer cancel() - owner, state, tester := createOwner4Test(ctx, c) + owner, state, tester := createOwner4Test(ctx, t) changefeedID := "test-changefeed" changefeedInfo := &model.ChangeFeedInfo{ @@ -89,7 +84,7 @@ func (s *ownerSuite) TestCreateRemoveChangefeed(c *check.C) { Config: config.GetDefaultReplicaConfig(), } changefeedStr, err := changefeedInfo.Marshal() - c.Assert(err, check.IsNil) + require.Nil(t, err) cdcKey := etcd.CDCKey{ Tp: etcd.CDCKeyTypeChangefeedInfo, ChangefeedID: changefeedID, @@ -97,29 +92,28 @@ func (s *ownerSuite) TestCreateRemoveChangefeed(c *check.C) { tester.MustUpdate(cdcKey.String(), []byte(changefeedStr)) _, err = owner.Tick(ctx, state) tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(owner.changefeeds, check.HasKey, changefeedID) + require.Nil(t, err) + require.Contains(t, owner.changefeeds, changefeedID) // delete changefeed info key to remove changefeed tester.MustUpdate(cdcKey.String(), nil) // this tick to clean the leak info fo the removed changefeed _, err = owner.Tick(ctx, state) - c.Assert(err, check.IsNil) + require.Nil(t, err) // this tick to remove the changefeed state in memory tester.MustApplyPatches() _, err = owner.Tick(ctx, state) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(owner.changefeeds, check.Not(check.HasKey), changefeedID) - c.Assert(state.Changefeeds, check.Not(check.HasKey), changefeedID) + + require.NotContains(t, owner.changefeeds, changefeedID) + require.NotContains(t, state.Changefeeds, changefeedID) tester.MustUpdate(cdcKey.String(), []byte(changefeedStr)) _, err = owner.Tick(ctx, state) tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(owner.changefeeds, check.HasKey, changefeedID) - + require.Nil(t, err) + require.Contains(t, owner.changefeeds, changefeedID) removeJob := model.AdminJob{ CfID: changefeedID, Type: model.AdminRemove, @@ -131,24 +125,23 @@ func (s *ownerSuite) TestCreateRemoveChangefeed(c *check.C) { mockedManager := &mockManager{Manager: owner.gcManager} owner.gcManager = mockedManager err = owner.gcManager.CheckStaleCheckpointTs(ctx, changefeedID, 0) - c.Assert(err, check.NotNil) + require.NotNil(t, err) // this tick create remove changefeed patches owner.EnqueueJob(removeJob) _, err = owner.Tick(ctx, state) - c.Assert(err, check.IsNil) + require.Nil(t, err) // apply patches and update owner's in memory changefeed states tester.MustApplyPatches() _, err = owner.Tick(ctx, state) - c.Assert(err, check.IsNil) - c.Assert(owner.changefeeds, check.Not(check.HasKey), changefeedID) + require.Nil(t, err) + require.NotContains(t, owner.changefeeds, changefeedID) } -func (s *ownerSuite) TestStopChangefeed(c *check.C) { - defer testleak.AfterTest(c)() +func TestStopChangefeed(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(false) - owner, state, tester := createOwner4Test(ctx, c) + owner, state, tester := createOwner4Test(ctx, t) ctx, cancel := cdcContext.WithCancel(ctx) defer cancel() @@ -158,7 +151,7 @@ func (s *ownerSuite) TestStopChangefeed(c *check.C) { Config: config.GetDefaultReplicaConfig(), } changefeedStr, err := changefeedInfo.Marshal() - c.Assert(err, check.IsNil) + require.Nil(t, err) cdcKey := etcd.CDCKey{ Tp: etcd.CDCKeyTypeChangefeedInfo, ChangefeedID: changefeedID, @@ -166,9 +159,8 @@ func (s *ownerSuite) TestStopChangefeed(c *check.C) { tester.MustUpdate(cdcKey.String(), []byte(changefeedStr)) _, err = owner.Tick(ctx, state) tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(owner.changefeeds, check.HasKey, changefeedID) - + require.Nil(t, err) + require.Contains(t, owner.changefeeds, changefeedID) // remove changefeed forcibly owner.EnqueueJob(model.AdminJob{ CfID: changefeedID, @@ -180,23 +172,20 @@ func (s *ownerSuite) TestStopChangefeed(c *check.C) { // this tick to clean the leak info fo the removed changefeed _, err = owner.Tick(ctx, state) - c.Assert(err, check.IsNil) - c.Assert(err, check.IsNil) + require.Nil(t, err) // this tick to remove the changefeed state in memory tester.MustApplyPatches() _, err = owner.Tick(ctx, state) - c.Assert(err, check.IsNil) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(owner.changefeeds, check.Not(check.HasKey), changefeedID) - c.Assert(state.Changefeeds, check.Not(check.HasKey), changefeedID) + require.Nil(t, err) + require.NotContains(t, owner.changefeeds, changefeedID) + require.NotContains(t, state.Changefeeds, changefeedID) } -func (s *ownerSuite) TestFixChangefeedState(c *check.C) { - defer testleak.AfterTest(c)() +func TestFixChangefeedState(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(false) - owner, state, tester := createOwner4Test(ctx, c) + owner, state, tester := createOwner4Test(ctx, t) // We need to do bootstrap. owner.bootstrapped = false changefeedID := "test-changefeed" @@ -208,7 +197,7 @@ func (s *ownerSuite) TestFixChangefeedState(c *check.C) { Config: config.GetDefaultReplicaConfig(), } changefeedStr, err := changefeedInfo.Marshal() - c.Assert(err, check.IsNil) + require.Nil(t, err) cdcKey := etcd.CDCKey{ Tp: etcd.CDCKeyTypeChangefeedInfo, ChangefeedID: changefeedID, @@ -217,23 +206,21 @@ func (s *ownerSuite) TestFixChangefeedState(c *check.C) { // For the first tick, we do a bootstrap, and it tries to fix the meta information. _, err = owner.Tick(ctx, state) tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(owner.bootstrapped, check.IsTrue) - c.Assert(owner.changefeeds, check.Not(check.HasKey), changefeedID) - + require.Nil(t, err) + require.True(t, owner.bootstrapped) + require.NotContains(t, owner.changefeeds, changefeedID) // Start tick normally. _, err = owner.Tick(ctx, state) tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(owner.changefeeds, check.HasKey, changefeedID) + require.Nil(t, err) + require.Contains(t, owner.changefeeds, changefeedID) // The meta information is fixed correctly. - c.Assert(owner.changefeeds[changefeedID].state.Info.State, check.Equals, model.StateStopped) + require.Equal(t, owner.changefeeds[changefeedID].state.Info.State, model.StateStopped) } -func (s *ownerSuite) TestFixChangefeedSinkProtocol(c *check.C) { - defer testleak.AfterTest(c)() +func TestFixChangefeedSinkProtocol(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(false) - owner, state, tester := createOwner4Test(ctx, c) + owner, state, tester := createOwner4Test(ctx, t) // We need to do bootstrap. owner.bootstrapped = false changefeedID := "test-changefeed" @@ -249,7 +236,7 @@ func (s *ownerSuite) TestFixChangefeedSinkProtocol(c *check.C) { }, } changefeedStr, err := changefeedInfo.Marshal() - c.Assert(err, check.IsNil) + require.Nil(t, err) cdcKey := etcd.CDCKey{ Tp: etcd.CDCKeyTypeChangefeedInfo, ChangefeedID: changefeedID, @@ -258,25 +245,23 @@ func (s *ownerSuite) TestFixChangefeedSinkProtocol(c *check.C) { // For the first tick, we do a bootstrap, and it tries to fix the meta information. _, err = owner.Tick(ctx, state) tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(owner.bootstrapped, check.IsTrue) - c.Assert(owner.changefeeds, check.Not(check.HasKey), changefeedID) + require.Nil(t, err) + require.True(t, owner.bootstrapped) + require.NotContains(t, owner.changefeeds, changefeedID) // Start tick normally. _, err = owner.Tick(ctx, state) tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(owner.changefeeds, check.HasKey, changefeedID) + require.Nil(t, err) + require.Contains(t, owner.changefeeds, changefeedID) // The meta information is fixed correctly. - c.Assert(owner.changefeeds[changefeedID].state.Info.SinkURI, - check.Equals, + require.Equal(t, owner.changefeeds[changefeedID].state.Info.SinkURI, "kafka://127.0.0.1:9092/ticdc-test2?protocol=open-protocol") } -func (s *ownerSuite) TestCheckClusterVersion(c *check.C) { - defer testleak.AfterTest(c)() +func TestCheckClusterVersion(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(false) - owner, state, tester := createOwner4Test(ctx, c) + owner, state, tester := createOwner4Test(ctx, t) ctx, cancel := cdcContext.WithCancel(ctx) defer cancel() @@ -288,7 +273,7 @@ func (s *ownerSuite) TestCheckClusterVersion(c *check.C) { Config: config.GetDefaultReplicaConfig(), } changefeedStr, err := changefeedInfo.Marshal() - c.Assert(err, check.IsNil) + require.Nil(t, err) cdcKey := etcd.CDCKey{ Tp: etcd.CDCKeyTypeChangefeedInfo, ChangefeedID: changefeedID, @@ -298,8 +283,8 @@ func (s *ownerSuite) TestCheckClusterVersion(c *check.C) { // check the tick is skipped and the changefeed will not be handled _, err = owner.Tick(ctx, state) tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(owner.changefeeds, check.Not(check.HasKey), changefeedID) + require.Nil(t, err) + require.NotContains(t, owner.changefeeds, changefeedID) tester.MustUpdate("/tidb/cdc/capture/6bbc01c8-0605-4f86-a0f9-b3119109b225", []byte(`{"id":"6bbc01c8-0605-4f86-a0f9-b3119109b225","address":"127.0.0.1:8300","version":"`+ctx.GlobalVars().CaptureInfo.Version+`"}`)) @@ -307,17 +292,16 @@ func (s *ownerSuite) TestCheckClusterVersion(c *check.C) { // check the tick is not skipped and the changefeed will be handled normally _, err = owner.Tick(ctx, state) tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(owner.changefeeds, check.HasKey, changefeedID) + require.Nil(t, err) + require.Contains(t, owner.changefeeds, changefeedID) } -func (s *ownerSuite) TestAdminJob(c *check.C) { - defer testleak.AfterTest(c)() +func TestAdminJob(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(false) ctx, cancel := cdcContext.WithCancel(ctx) defer cancel() - owner, _, _ := createOwner4Test(ctx, c) + owner, _, _ := createOwner4Test(ctx, t) owner.EnqueueJob(model.AdminJob{ CfID: "test-changefeed1", Type: model.AdminResume, @@ -330,11 +314,11 @@ func (s *ownerSuite) TestAdminJob(c *check.C) { // remove job.done, it's hard to check deep equals jobs := owner.takeOwnerJobs() for _, job := range jobs { - c.Assert(job.done, check.NotNil) + require.NotNil(t, job.done) close(job.done) job.done = nil } - c.Assert(jobs, check.DeepEquals, []*ownerJob{ + require.Equal(t, jobs, []*ownerJob{ { tp: ownerJobTypeAdminJob, adminJob: &model.AdminJob{ @@ -355,11 +339,10 @@ func (s *ownerSuite) TestAdminJob(c *check.C) { debugInfoWriter: &buf, }, }) - c.Assert(owner.takeOwnerJobs(), check.HasLen, 0) + require.Len(t, owner.takeOwnerJobs(), 0) } -func (s *ownerSuite) TestUpdateGCSafePoint(c *check.C) { - defer testleak.AfterTest(c)() +func TestUpdateGCSafePoint(t *testing.T) { mockPDClient := &gc.MockPDClient{} o := NewOwner(mockPDClient) o.gcManager = gc.NewManager(mockPDClient) @@ -367,23 +350,23 @@ func (s *ownerSuite) TestUpdateGCSafePoint(c *check.C) { ctx, cancel := cdcContext.WithCancel(ctx) defer cancel() state := orchestrator.NewGlobalState() - tester := orchestrator.NewReactorStateTester(c, state, nil) + tester := orchestrator.NewReactorStateTester(t, state, nil) // no changefeed, the gc safe point should be max uint64 mockPDClient.UpdateServiceGCSafePointFunc = func(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { // Owner will do a snapshot read at (checkpointTs - 1) from TiKV, // set GC safepoint to (checkpointTs - 1) - c.Assert(safePoint, check.Equals, uint64(math.MaxUint64-1)) + require.Equal(t, safePoint, uint64(math.MaxUint64-1)) return 0, nil } err := o.updateGCSafepoint(ctx, state) - c.Assert(err, check.IsNil) + require.Nil(t, err) // add a failed changefeed, it must not trigger update GC safepoint. mockPDClient.UpdateServiceGCSafePointFunc = func(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { - c.Fatal("must not update") + t.Fatal("must not update") return 0, nil } changefeedID1 := "changefeed-test1" @@ -397,7 +380,7 @@ func (s *ownerSuite) TestUpdateGCSafePoint(c *check.C) { }) tester.MustApplyPatches() err = o.updateGCSafepoint(ctx, state) - c.Assert(err, check.IsNil) + require.Nil(t, err) // switch the state of changefeed to normal, it must update GC safepoint to // 1 (checkpoint Ts of changefeed-test1). @@ -406,8 +389,8 @@ func (s *ownerSuite) TestUpdateGCSafePoint(c *check.C) { func(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { // Owner will do a snapshot read at (checkpointTs - 1) from TiKV, // set GC safepoint to (checkpointTs - 1) - c.Assert(safePoint, check.Equals, uint64(1)) - c.Assert(serviceID, check.Equals, gc.CDCServiceSafePointID) + require.Equal(t, safePoint, uint64(1)) + require.Equal(t, serviceID, gc.CDCServiceSafePointID) ch <- struct{}{} return 0, nil } @@ -418,10 +401,10 @@ func (s *ownerSuite) TestUpdateGCSafePoint(c *check.C) { }) tester.MustApplyPatches() err = o.updateGCSafepoint(ctx, state) - c.Assert(err, check.IsNil) + require.Nil(t, err) select { case <-time.After(5 * time.Second): - c.Fatal("timeout") + t.Fatal("timeout") case <-ch: } @@ -444,28 +427,27 @@ func (s *ownerSuite) TestUpdateGCSafePoint(c *check.C) { func(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { // Owner will do a snapshot read at (checkpointTs - 1) from TiKV, // set GC safepoint to (checkpointTs - 1) - c.Assert(safePoint, check.Equals, uint64(19)) - c.Assert(serviceID, check.Equals, gc.CDCServiceSafePointID) + require.Equal(t, safePoint, uint64(19)) + require.Equal(t, serviceID, gc.CDCServiceSafePointID) ch <- struct{}{} return 0, nil } err = o.updateGCSafepoint(ctx, state) - c.Assert(err, check.IsNil) + require.Nil(t, err) select { case <-time.After(5 * time.Second): - c.Fatal("timeout") + t.Fatal("timeout") case <-ch: } } // make sure handleJobs works well even if there is two different // version of captures in the cluster -func (s *ownerSuite) TestHandleJobsDontBlock(c *check.C) { - defer testleak.AfterTest(c)() +func TestHandleJobsDontBlock(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(false) ctx, cancel := cdcContext.WithCancel(ctx) defer cancel() - owner, state, tester := createOwner4Test(ctx, c) + owner, state, tester := createOwner4Test(ctx, t) statusProvider := owner.StatusProvider() // work well @@ -476,7 +458,7 @@ func (s *ownerSuite) TestHandleJobsDontBlock(c *check.C) { State: model.StateNormal, } changefeedStr, err := cfInfo1.Marshal() - c.Assert(err, check.IsNil) + require.Nil(t, err) cdcKey := etcd.CDCKey{ Tp: etcd.CDCKeyTypeChangefeedInfo, ChangefeedID: cf1, @@ -484,9 +466,9 @@ func (s *ownerSuite) TestHandleJobsDontBlock(c *check.C) { tester.MustUpdate(cdcKey.String(), []byte(changefeedStr)) _, err = owner.Tick(ctx, state) tester.MustApplyPatches() - c.Assert(err, check.IsNil) + require.Nil(t, err) - c.Assert(owner.changefeeds, check.HasKey, cf1) + require.Contains(t, owner.changefeeds, cf1) // add an non-consistent version capture captureInfo := &model.CaptureInfo{ @@ -499,7 +481,7 @@ func (s *ownerSuite) TestHandleJobsDontBlock(c *check.C) { CaptureID: captureInfo.ID, } v, err := captureInfo.Marshal() - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustUpdate(cdcKey.String(), v) // try to add another changefeed @@ -510,7 +492,7 @@ func (s *ownerSuite) TestHandleJobsDontBlock(c *check.C) { State: model.StateNormal, } changefeedStr1, err := cfInfo2.Marshal() - c.Assert(err, check.IsNil) + require.Nil(t, err) cdcKey = etcd.CDCKey{ Tp: etcd.CDCKeyTypeChangefeedInfo, ChangefeedID: cf2, @@ -518,10 +500,10 @@ func (s *ownerSuite) TestHandleJobsDontBlock(c *check.C) { tester.MustUpdate(cdcKey.String(), []byte(changefeedStr1)) _, err = owner.Tick(ctx, state) tester.MustApplyPatches() - c.Assert(err, check.IsNil) + require.Nil(t, err) // make sure this changefeed add failed, which means that owner are return // in clusterVersionConsistent check - c.Assert(owner.changefeeds[cf2], check.IsNil) + require.Nil(t, owner.changefeeds[cf2]) // make sure statusProvider works well ctx1, cancel := context.WithTimeout(context.Background(), time.Second*5) @@ -543,13 +525,13 @@ WorkLoop: case <-done: break WorkLoop case <-ctx1.Done(): - c.Fatal(ctx1.Err()) + t.Fatal(ctx1.Err()) case <-ticker.C: _, err = owner.Tick(ctx, state) - c.Assert(err, check.IsNil) + require.Nil(t, err) } } - c.Assert(errIn, check.IsNil) - c.Assert(infos[cf1], check.NotNil) - c.Assert(infos[cf2], check.IsNil) + require.Nil(t, errIn) + require.NotNil(t, infos[cf1]) + require.Nil(t, infos[cf2]) } diff --git a/cdc/owner/scheduler_v1_test.go b/cdc/owner/scheduler_v1_test.go index eb2950b7190..289c951b3db 100644 --- a/cdc/owner/scheduler_v1_test.go +++ b/cdc/owner/scheduler_v1_test.go @@ -16,17 +16,15 @@ package owner import ( "fmt" "math/rand" + "testing" - "github.com/pingcap/check" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/orchestrator" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) -var _ = check.Suite(&schedulerSuite{}) - -type schedulerSuite struct { +type schedulerTester struct { changefeedID model.ChangeFeedID state *orchestrator.ChangefeedReactorState tester *orchestrator.ReactorStateTester @@ -34,10 +32,10 @@ type schedulerSuite struct { scheduler *oldScheduler } -func (s *schedulerSuite) reset(c *check.C) { +func (s *schedulerTester) reset(t *testing.T) { s.changefeedID = fmt.Sprintf("test-changefeed-%x", rand.Uint32()) s.state = orchestrator.NewChangefeedReactorState("test-changefeed") - s.tester = orchestrator.NewReactorStateTester(c, s.state, nil) + s.tester = orchestrator.NewReactorStateTester(t, s.state, nil) s.scheduler = newSchedulerV1().(*schedulerV1CompatWrapper).inner s.captures = make(map[model.CaptureID]*model.CaptureInfo) s.state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { @@ -46,7 +44,7 @@ func (s *schedulerSuite) reset(c *check.C) { s.tester.MustApplyPatches() } -func (s *schedulerSuite) addCapture(captureID model.CaptureID) { +func (s *schedulerTester) addCapture(captureID model.CaptureID) { captureInfo := &model.CaptureInfo{ ID: captureID, } @@ -57,7 +55,7 @@ func (s *schedulerSuite) addCapture(captureID model.CaptureID) { s.tester.MustApplyPatches() } -func (s *schedulerSuite) finishTableOperation(captureID model.CaptureID, tableIDs ...model.TableID) { +func (s *schedulerTester) finishTableOperation(captureID model.CaptureID, tableIDs ...model.TableID) { s.state.PatchTaskStatus(captureID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { for _, tableID := range tableIDs { status.Operation[tableID].Status = model.OperFinished @@ -82,10 +80,9 @@ func (s *schedulerSuite) finishTableOperation(captureID model.CaptureID, tableID s.tester.MustApplyPatches() } -func (s *schedulerSuite) TestScheduleOneCapture(c *check.C) { - defer testleak.AfterTest(c)() - - s.reset(c) +func TestScheduleOneCapture(t *testing.T) { + s := &schedulerTester{} + s.reset(t) captureID := "test-capture-0" s.addCapture(captureID) @@ -100,27 +97,27 @@ func (s *schedulerSuite) TestScheduleOneCapture(c *check.C) { s.tester.MustUpdate(key.String(), nil) s.tester.MustApplyPatches() - s.reset(c) + s.reset(t) captureID = "test-capture-1" s.addCapture(captureID) // add three tables shouldUpdateState, err := s.scheduler.Tick(s.state, []model.TableID{1, 2, 3, 4}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsFalse) + require.Nil(t, err) + require.False(t, shouldUpdateState) s.tester.MustApplyPatches() - c.Assert(s.state.TaskStatuses[captureID].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID].Tables, map[model.TableID]*model.TableReplicaInfo{ 1: {StartTs: 0}, 2: {StartTs: 0}, 3: {StartTs: 0}, 4: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + require.Equal(t, s.state.TaskStatuses[captureID].Operation, map[model.TableID]*model.TableOperation{ 1: {Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, 2: {Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, 3: {Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, 4: {Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, }) shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1, 2, 3, 4}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsTrue) + require.Nil(t, err) + require.True(t, shouldUpdateState) s.tester.MustApplyPatches() // two tables finish adding operation @@ -128,13 +125,13 @@ func (s *schedulerSuite) TestScheduleOneCapture(c *check.C) { // remove table 1,2 and add table 4,5 shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{3, 4, 5}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsFalse) + require.Nil(t, err) + require.False(t, shouldUpdateState) s.tester.MustApplyPatches() - c.Assert(s.state.TaskStatuses[captureID].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID].Tables, map[model.TableID]*model.TableReplicaInfo{ 3: {StartTs: 0}, 4: {StartTs: 0}, 5: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + require.Equal(t, s.state.TaskStatuses[captureID].Operation, map[model.TableID]*model.TableOperation{ 1: {Delete: true, BoundaryTs: 0, Status: model.OperDispatched}, 2: {Delete: true, BoundaryTs: 0, Status: model.OperDispatched}, 4: {Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, @@ -147,13 +144,13 @@ func (s *schedulerSuite) TestScheduleOneCapture(c *check.C) { s.scheduler.MoveTable(3, "fake-capture") s.scheduler.MoveTable(4, "fake-capture") shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{3, 4, 5}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsFalse) + require.Nil(t, err) + require.False(t, shouldUpdateState) s.tester.MustApplyPatches() - c.Assert(s.state.TaskStatuses[captureID].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID].Tables, map[model.TableID]*model.TableReplicaInfo{ 4: {StartTs: 0}, 5: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + require.Equal(t, s.state.TaskStatuses[captureID].Operation, map[model.TableID]*model.TableOperation{ 1: {Delete: true, BoundaryTs: 0, Status: model.OperDispatched}, 2: {Delete: true, BoundaryTs: 0, Status: model.OperDispatched}, 3: {Delete: true, BoundaryTs: 0, Status: model.OperDispatched}, @@ -165,77 +162,77 @@ func (s *schedulerSuite) TestScheduleOneCapture(c *check.C) { s.finishTableOperation(captureID, 1, 2, 3, 4, 5) shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{3, 4, 5}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsTrue) + require.Nil(t, err) + require.True(t, shouldUpdateState) s.tester.MustApplyPatches() - c.Assert(s.state.TaskStatuses[captureID].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID].Tables, map[model.TableID]*model.TableReplicaInfo{ 4: {StartTs: 0}, 5: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{}) + require.Equal(t, s.state.TaskStatuses[captureID].Operation, map[model.TableID]*model.TableOperation{}) // table 3 is missing by expected, because the table was trying to move to a invalid capture // and the move will failed, the table 3 will be add in next tick shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{3, 4, 5}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsFalse) + require.Nil(t, err) + require.False(t, shouldUpdateState) s.tester.MustApplyPatches() - c.Assert(s.state.TaskStatuses[captureID].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID].Tables, map[model.TableID]*model.TableReplicaInfo{ 4: {StartTs: 0}, 5: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{}) + require.Equal(t, s.state.TaskStatuses[captureID].Operation, map[model.TableID]*model.TableOperation{}) shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{3, 4, 5}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsFalse) + require.Nil(t, err) + require.False(t, shouldUpdateState) s.tester.MustApplyPatches() - c.Assert(s.state.TaskStatuses[captureID].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID].Tables, map[model.TableID]*model.TableReplicaInfo{ 3: {StartTs: 0}, 4: {StartTs: 0}, 5: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + require.Equal(t, s.state.TaskStatuses[captureID].Operation, map[model.TableID]*model.TableOperation{ 3: {Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, }) } -func (s *schedulerSuite) TestScheduleMoveTable(c *check.C) { - defer testleak.AfterTest(c)() - s.reset(c) +func TestScheduleMoveTable(t *testing.T) { + s := &schedulerTester{} + s.reset(t) captureID1 := "test-capture-1" captureID2 := "test-capture-2" s.addCapture(captureID1) // add a table shouldUpdateState, err := s.scheduler.Tick(s.state, []model.TableID{1}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsFalse) + require.Nil(t, err) + require.False(t, shouldUpdateState) s.tester.MustApplyPatches() - c.Assert(s.state.TaskStatuses[captureID1].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID1].Tables, map[model.TableID]*model.TableReplicaInfo{ 1: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID1].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + require.Equal(t, s.state.TaskStatuses[captureID1].Operation, map[model.TableID]*model.TableOperation{ 1: {Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, }) s.finishTableOperation(captureID1, 1) shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsTrue) + require.Nil(t, err) + require.True(t, shouldUpdateState) s.tester.MustApplyPatches() s.addCapture(captureID2) // add a table shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1, 2}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsFalse) + require.Nil(t, err) + require.False(t, shouldUpdateState) s.tester.MustApplyPatches() - c.Assert(s.state.TaskStatuses[captureID1].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID1].Tables, map[model.TableID]*model.TableReplicaInfo{ 1: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID1].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{}) - c.Assert(s.state.TaskStatuses[captureID2].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID1].Operation, map[model.TableID]*model.TableOperation{}) + require.Equal(t, s.state.TaskStatuses[captureID2].Tables, map[model.TableID]*model.TableReplicaInfo{ 2: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID2].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + require.Equal(t, s.state.TaskStatuses[captureID2].Operation, map[model.TableID]*model.TableOperation{ 2: {Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, }) @@ -243,48 +240,48 @@ func (s *schedulerSuite) TestScheduleMoveTable(c *check.C) { s.scheduler.MoveTable(2, captureID1) shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1, 2}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsFalse) + require.Nil(t, err) + require.False(t, shouldUpdateState) s.tester.MustApplyPatches() - c.Assert(s.state.TaskStatuses[captureID1].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID1].Tables, map[model.TableID]*model.TableReplicaInfo{ 1: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID1].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{}) - c.Assert(s.state.TaskStatuses[captureID2].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{}) - c.Assert(s.state.TaskStatuses[captureID2].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + require.Equal(t, s.state.TaskStatuses[captureID1].Operation, map[model.TableID]*model.TableOperation{}) + require.Equal(t, s.state.TaskStatuses[captureID2].Tables, map[model.TableID]*model.TableReplicaInfo{}) + require.Equal(t, s.state.TaskStatuses[captureID2].Operation, map[model.TableID]*model.TableOperation{ 2: {Delete: true, BoundaryTs: 0, Status: model.OperDispatched}, }) s.finishTableOperation(captureID2, 2) shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1, 2}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsTrue) + require.Nil(t, err) + require.True(t, shouldUpdateState) s.tester.MustApplyPatches() - c.Assert(s.state.TaskStatuses[captureID1].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID1].Tables, map[model.TableID]*model.TableReplicaInfo{ 1: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID1].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{}) - c.Assert(s.state.TaskStatuses[captureID2].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{}) - c.Assert(s.state.TaskStatuses[captureID2].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{}) + require.Equal(t, s.state.TaskStatuses[captureID1].Operation, map[model.TableID]*model.TableOperation{}) + require.Equal(t, s.state.TaskStatuses[captureID2].Tables, map[model.TableID]*model.TableReplicaInfo{}) + require.Equal(t, s.state.TaskStatuses[captureID2].Operation, map[model.TableID]*model.TableOperation{}) shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1, 2}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsFalse) + require.Nil(t, err) + require.False(t, shouldUpdateState) s.tester.MustApplyPatches() - c.Assert(s.state.TaskStatuses[captureID1].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{ + require.Equal(t, s.state.TaskStatuses[captureID1].Tables, map[model.TableID]*model.TableReplicaInfo{ 1: {StartTs: 0}, 2: {StartTs: 0}, }) - c.Assert(s.state.TaskStatuses[captureID1].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{ + require.Equal(t, s.state.TaskStatuses[captureID1].Operation, map[model.TableID]*model.TableOperation{ 2: {Delete: false, BoundaryTs: 0, Status: model.OperDispatched}, }) - c.Assert(s.state.TaskStatuses[captureID2].Tables, check.DeepEquals, map[model.TableID]*model.TableReplicaInfo{}) - c.Assert(s.state.TaskStatuses[captureID2].Operation, check.DeepEquals, map[model.TableID]*model.TableOperation{}) + require.Equal(t, s.state.TaskStatuses[captureID2].Tables, map[model.TableID]*model.TableReplicaInfo{}) + require.Equal(t, s.state.TaskStatuses[captureID2].Operation, map[model.TableID]*model.TableOperation{}) } -func (s *schedulerSuite) TestScheduleRebalance(c *check.C) { - defer testleak.AfterTest(c)() - s.reset(c) +func TestScheduleRebalance(t *testing.T) { + s := &schedulerTester{} + s.reset(t) captureID1 := "test-capture-1" captureID2 := "test-capture-2" captureID3 := "test-capture-3" @@ -306,13 +303,13 @@ func (s *schedulerSuite) TestScheduleRebalance(c *check.C) { // rebalance table shouldUpdateState, err := s.scheduler.Tick(s.state, []model.TableID{1, 2, 3, 4, 5, 6}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsFalse) + require.Nil(t, err) + require.False(t, shouldUpdateState) s.tester.MustApplyPatches() // 4 tables remove in capture 1, this 4 tables will be added to another capture in next tick - c.Assert(s.state.TaskStatuses[captureID1].Tables, check.HasLen, 2) - c.Assert(s.state.TaskStatuses[captureID2].Tables, check.HasLen, 0) - c.Assert(s.state.TaskStatuses[captureID3].Tables, check.HasLen, 0) + require.Len(t, s.state.TaskStatuses[captureID1].Tables, 2) + require.Len(t, s.state.TaskStatuses[captureID2].Tables, 0) + require.Len(t, s.state.TaskStatuses[captureID3].Tables, 0) s.state.PatchTaskStatus(captureID1, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { for _, opt := range status.Operation { @@ -321,7 +318,7 @@ func (s *schedulerSuite) TestScheduleRebalance(c *check.C) { return status, true, nil }) s.state.PatchTaskWorkload(captureID1, func(workload model.TaskWorkload) (model.TaskWorkload, bool, error) { - c.Assert(workload, check.IsNil) + require.Nil(t, workload) workload = make(model.TaskWorkload) for tableID := range s.state.TaskStatuses[captureID1].Tables { workload[tableID] = model.WorkloadInfo{Workload: 1} @@ -332,26 +329,26 @@ func (s *schedulerSuite) TestScheduleRebalance(c *check.C) { // clean finished operation shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1, 2, 3, 4, 5, 6}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsTrue) + require.Nil(t, err) + require.True(t, shouldUpdateState) s.tester.MustApplyPatches() // 4 tables add to another capture in this tick - c.Assert(s.state.TaskStatuses[captureID1].Operation, check.HasLen, 0) + require.Len(t, s.state.TaskStatuses[captureID1].Operation, 0) // rebalance table shouldUpdateState, err = s.scheduler.Tick(s.state, []model.TableID{1, 2, 3, 4, 5, 6}, s.captures) - c.Assert(err, check.IsNil) - c.Assert(shouldUpdateState, check.IsFalse) + require.Nil(t, err) + require.False(t, shouldUpdateState) s.tester.MustApplyPatches() // 4 tables add to another capture in this tick - c.Assert(s.state.TaskStatuses[captureID1].Tables, check.HasLen, 2) - c.Assert(s.state.TaskStatuses[captureID2].Tables, check.HasLen, 2) - c.Assert(s.state.TaskStatuses[captureID3].Tables, check.HasLen, 2) + require.Len(t, s.state.TaskStatuses[captureID1].Tables, 2) + require.Len(t, s.state.TaskStatuses[captureID2].Tables, 2) + require.Len(t, s.state.TaskStatuses[captureID3].Tables, 2) tableIDs := make(map[model.TableID]struct{}) for _, status := range s.state.TaskStatuses { for tableID := range status.Tables { tableIDs[tableID] = struct{}{} } } - c.Assert(tableIDs, check.DeepEquals, map[model.TableID]struct{}{1: {}, 2: {}, 3: {}, 4: {}, 5: {}, 6: {}}) + require.Equal(t, tableIDs, map[model.TableID]struct{}{1: {}, 2: {}, 3: {}, 4: {}, 5: {}, 6: {}}) } diff --git a/cdc/owner/schema_test.go b/cdc/owner/schema_test.go index 9ed7e8ec14f..9c8abb71316 100644 --- a/cdc/owner/schema_test.go +++ b/cdc/owner/schema_test.go @@ -15,38 +15,33 @@ package owner import ( "sort" + "testing" - "github.com/pingcap/check" timodel "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" ) -var _ = check.Suite(&schemaSuite{}) - -type schemaSuite struct{} - -func (s *schemaSuite) TestAllPhysicalTables(c *check.C) { - defer testleak.AfterTest(c)() - helper := entry.NewSchemaTestHelper(c) +func TestAllPhysicalTables(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) defer helper.Close() ver, err := helper.Storage().CurrentVersion(oracle.GlobalTxnScope) - c.Assert(err, check.IsNil) + require.Nil(t, err) schema, err := newSchemaWrap4Owner(helper.Storage(), ver.Ver, config.GetDefaultReplicaConfig()) - c.Assert(err, check.IsNil) - c.Assert(schema.AllPhysicalTables(), check.HasLen, 0) + require.Nil(t, err) + require.Len(t, schema.AllPhysicalTables(), 0) // add normal table job := helper.DDL2Job("create table test.t1(id int primary key)") tableIDT1 := job.BinlogInfo.TableInfo.ID - c.Assert(schema.HandleDDL(job), check.IsNil) - c.Assert(schema.AllPhysicalTables(), check.DeepEquals, []model.TableID{tableIDT1}) + require.Nil(t, schema.HandleDDL(job)) + require.Equal(t, schema.AllPhysicalTables(), []model.TableID{tableIDT1}) // add ineligible table - c.Assert(schema.HandleDDL(helper.DDL2Job("create table test.t2(id int)")), check.IsNil) - c.Assert(schema.AllPhysicalTables(), check.DeepEquals, []model.TableID{tableIDT1}) + require.Nil(t, schema.HandleDDL(helper.DDL2Job("create table test.t2(id int)"))) + require.Equal(t, schema.AllPhysicalTables(), []model.TableID{tableIDT1}) // add partition table job = helper.DDL2Job(`CREATE TABLE test.employees ( id INT NOT NULL AUTO_INCREMENT PRIMARY KEY, @@ -62,7 +57,7 @@ func (s *schemaSuite) TestAllPhysicalTables(c *check.C) { PARTITION p2 VALUES LESS THAN (15), PARTITION p3 VALUES LESS THAN (20) )`) - c.Assert(schema.HandleDDL(job), check.IsNil) + require.Nil(t, schema.HandleDDL(job)) expectedTableIDs := []model.TableID{tableIDT1} for _, p := range job.BinlogInfo.TableInfo.GetPartitionInfo().Definitions { expectedTableIDs = append(expectedTableIDs, p.ID) @@ -74,42 +69,41 @@ func (s *schemaSuite) TestAllPhysicalTables(c *check.C) { } sortTableIDs(expectedTableIDs) sortTableIDs(schema.AllPhysicalTables()) - c.Assert(schema.AllPhysicalTables(), check.DeepEquals, expectedTableIDs) + require.Equal(t, schema.AllPhysicalTables(), expectedTableIDs) } -func (s *schemaSuite) TestIsIneligibleTableID(c *check.C) { - defer testleak.AfterTest(c)() - helper := entry.NewSchemaTestHelper(c) +func TestIsIneligibleTableID(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) defer helper.Close() ver, err := helper.Storage().CurrentVersion(oracle.GlobalTxnScope) - c.Assert(err, check.IsNil) + require.Nil(t, err) schema, err := newSchemaWrap4Owner(helper.Storage(), ver.Ver, config.GetDefaultReplicaConfig()) - c.Assert(err, check.IsNil) + require.Nil(t, err) // add normal table job := helper.DDL2Job("create table test.t1(id int primary key)") tableIDT1 := job.BinlogInfo.TableInfo.ID - c.Assert(schema.HandleDDL(job), check.IsNil) + require.Nil(t, schema.HandleDDL(job)) // add ineligible table job = helper.DDL2Job("create table test.t2(id int)") tableIDT2 := job.BinlogInfo.TableInfo.ID - c.Assert(schema.HandleDDL(job), check.IsNil) - c.Assert(schema.IsIneligibleTableID(tableIDT1), check.IsFalse) - c.Assert(schema.IsIneligibleTableID(tableIDT2), check.IsTrue) + + require.Nil(t, schema.HandleDDL(job)) + require.False(t, schema.IsIneligibleTableID(tableIDT1)) + require.True(t, schema.IsIneligibleTableID(tableIDT2)) } -func (s *schemaSuite) TestBuildDDLEvent(c *check.C) { - defer testleak.AfterTest(c)() - helper := entry.NewSchemaTestHelper(c) +func TestBuildDDLEvent(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) defer helper.Close() ver, err := helper.Storage().CurrentVersion(oracle.GlobalTxnScope) - c.Assert(err, check.IsNil) + require.Nil(t, err) schema, err := newSchemaWrap4Owner(helper.Storage(), ver.Ver, config.GetDefaultReplicaConfig()) - c.Assert(err, check.IsNil) + require.Nil(t, err) // add normal table job := helper.DDL2Job("create table test.t1(id int primary key)") event, err := schema.BuildDDLEvent(job) - c.Assert(err, check.IsNil) - c.Assert(event, check.DeepEquals, &model.DDLEvent{ + require.Nil(t, err) + require.Equal(t, event, &model.DDLEvent{ StartTs: job.StartTS, CommitTs: job.BinlogInfo.FinishedTS, Query: "create table test.t1(id int primary key)", @@ -122,11 +116,11 @@ func (s *schemaSuite) TestBuildDDLEvent(c *check.C) { }, PreTableInfo: nil, }) - c.Assert(schema.HandleDDL(job), check.IsNil) + require.Nil(t, schema.HandleDDL(job)) job = helper.DDL2Job("ALTER TABLE test.t1 ADD COLUMN c1 CHAR(16) NOT NULL") event, err = schema.BuildDDLEvent(job) - c.Assert(err, check.IsNil) - c.Assert(event, check.DeepEquals, &model.DDLEvent{ + require.Nil(t, err) + require.Equal(t, event, &model.DDLEvent{ StartTs: job.StartTS, CommitTs: job.BinlogInfo.FinishedTS, Query: "ALTER TABLE test.t1 ADD COLUMN c1 CHAR(16) NOT NULL", @@ -146,22 +140,21 @@ func (s *schemaSuite) TestBuildDDLEvent(c *check.C) { }) } -func (s *schemaSuite) TestSinkTableInfos(c *check.C) { - defer testleak.AfterTest(c)() - helper := entry.NewSchemaTestHelper(c) +func TestSinkTableInfos(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) defer helper.Close() ver, err := helper.Storage().CurrentVersion(oracle.GlobalTxnScope) - c.Assert(err, check.IsNil) + require.Nil(t, err) schema, err := newSchemaWrap4Owner(helper.Storage(), ver.Ver, config.GetDefaultReplicaConfig()) - c.Assert(err, check.IsNil) + require.Nil(t, err) // add normal table job := helper.DDL2Job("create table test.t1(id int primary key)") tableIDT1 := job.BinlogInfo.TableInfo.ID - c.Assert(schema.HandleDDL(job), check.IsNil) + require.Nil(t, schema.HandleDDL(job)) // add ineligible table job = helper.DDL2Job("create table test.t2(id int)") - c.Assert(schema.HandleDDL(job), check.IsNil) - c.Assert(schema.SinkTableInfos(), check.DeepEquals, []*model.SimpleTableInfo{ + require.Nil(t, schema.HandleDDL(job)) + require.Equal(t, schema.SinkTableInfos(), []*model.SimpleTableInfo{ { Schema: "test", Table: "t1", diff --git a/cdc/processor/main_test.go b/cdc/processor/main_test.go new file mode 100644 index 00000000000..5d7ddb7ad5e --- /dev/null +++ b/cdc/processor/main_test.go @@ -0,0 +1,24 @@ +// Copyright 2022 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 processor + +import ( + "testing" + + "github.com/pingcap/tiflow/pkg/leakutil" +) + +func TestMain(m *testing.M) { + leakutil.SetUpLeakTest(m) +} diff --git a/cdc/processor/manager_test.go b/cdc/processor/manager_test.go index 075fbca5fc3..8415af305aa 100644 --- a/cdc/processor/manager_test.go +++ b/cdc/processor/manager_test.go @@ -17,9 +17,9 @@ import ( "bytes" "fmt" "math" + "testing" "time" - "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/tiflow/cdc/model" tablepipeline "github.com/pingcap/tiflow/cdc/processor/pipeline" @@ -27,31 +27,29 @@ import ( cdcContext "github.com/pingcap/tiflow/pkg/context" cerrors "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/orchestrator" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) -type managerSuite struct { +type managerTester struct { manager *Manager state *orchestrator.GlobalReactorState tester *orchestrator.ReactorStateTester } -var _ = check.Suite(&managerSuite{}) - // NewManager4Test creates a new processor manager for test func NewManager4Test( - c *check.C, + t *testing.T, createTablePipeline func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error), ) *Manager { m := NewManager() m.newProcessor = func(ctx cdcContext.Context) *processor { - return newProcessor4Test(ctx, c, createTablePipeline) + return newProcessor4Test(ctx, t, createTablePipeline) } return m } -func (s *managerSuite) resetSuit(ctx cdcContext.Context, c *check.C) { - s.manager = NewManager4Test(c, func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) { +func (s *managerTester) resetSuit(ctx cdcContext.Context, t *testing.T) { + s.manager = NewManager4Test(t, func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) { return &mockTablePipeline{ tableID: tableID, name: fmt.Sprintf("`test`.`table%d`", tableID), @@ -62,28 +60,28 @@ func (s *managerSuite) resetSuit(ctx cdcContext.Context, c *check.C) { }) s.state = orchestrator.NewGlobalState() captureInfoBytes, err := ctx.GlobalVars().CaptureInfo.Marshal() - c.Assert(err, check.IsNil) - s.tester = orchestrator.NewReactorStateTester(c, s.state, map[string]string{ + require.Nil(t, err) + s.tester = orchestrator.NewReactorStateTester(t, s.state, map[string]string{ fmt.Sprintf("/tidb/cdc/capture/%s", ctx.GlobalVars().CaptureInfo.ID): string(captureInfoBytes), }) } -func (s *managerSuite) TestChangefeed(c *check.C) { - defer testleak.AfterTest(c)() +func TestChangefeed(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(false) - s.resetSuit(ctx, c) + s := &managerTester{} + s.resetSuit(ctx, t) var err error // no changefeed _, err = s.manager.Tick(ctx, s.state) - c.Assert(err, check.IsNil) + require.Nil(t, err) // an inactive changefeed s.state.Changefeeds["test-changefeed"] = orchestrator.NewChangefeedReactorState("test-changefeed") _, err = s.manager.Tick(ctx, s.state) s.tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(s.manager.processors, check.HasLen, 0) + require.Nil(t, err) + require.Len(t, s.manager.processors, 0) // an active changefeed s.state.Changefeeds["test-changefeed"].PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { @@ -106,8 +104,8 @@ func (s *managerSuite) TestChangefeed(c *check.C) { s.tester.MustApplyPatches() _, err = s.manager.Tick(ctx, s.state) s.tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(s.manager.processors, check.HasLen, 1) + require.Nil(t, err) + require.Len(t, s.manager.processors, 1) // processor return errors s.state.Changefeeds["test-changefeed"].PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { @@ -121,19 +119,19 @@ func (s *managerSuite) TestChangefeed(c *check.C) { s.tester.MustApplyPatches() _, err = s.manager.Tick(ctx, s.state) s.tester.MustApplyPatches() - c.Assert(err, check.IsNil) - c.Assert(s.manager.processors, check.HasLen, 0) + require.Nil(t, err) + require.Len(t, s.manager.processors, 0) } -func (s *managerSuite) TestDebugInfo(c *check.C) { - defer testleak.AfterTest(c)() +func TestDebugInfo(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(false) - s.resetSuit(ctx, c) + s := &managerTester{} + s.resetSuit(ctx, t) var err error // no changefeed _, err = s.manager.Tick(ctx, s.state) - c.Assert(err, check.IsNil) + require.Nil(t, err) // an active changefeed s.state.Changefeeds["test-changefeed"] = orchestrator.NewChangefeedReactorState("test-changefeed") @@ -156,38 +154,38 @@ func (s *managerSuite) TestDebugInfo(c *check.C) { }) s.tester.MustApplyPatches() _, err = s.manager.Tick(ctx, s.state) - c.Assert(err, check.IsNil) + require.Nil(t, err) s.tester.MustApplyPatches() - c.Assert(s.manager.processors, check.HasLen, 1) + require.Len(t, s.manager.processors, 1) done := make(chan struct{}) go func() { defer close(done) for { _, err = s.manager.Tick(ctx, s.state) if err != nil { - c.Assert(cerrors.ErrReactorFinished.Equal(errors.Cause(err)), check.IsTrue) + require.True(t, cerrors.ErrReactorFinished.Equal(errors.Cause(err))) return } - c.Assert(err, check.IsNil) + require.Nil(t, err) s.tester.MustApplyPatches() } }() buf := bytes.NewBufferString("") s.manager.WriteDebugInfo(buf) - c.Assert(len(buf.String()), check.Greater, 0) + require.Greater(t, len(buf.String()), 0) s.manager.AsyncClose() <-done } -func (s *managerSuite) TestClose(c *check.C) { - defer testleak.AfterTest(c)() +func TestClose(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(false) - s.resetSuit(ctx, c) + s := &managerTester{} + s.resetSuit(ctx, t) var err error // no changefeed _, err = s.manager.Tick(ctx, s.state) - c.Assert(err, check.IsNil) + require.Nil(t, err) // an active changefeed s.state.Changefeeds["test-changefeed"] = orchestrator.NewChangefeedReactorState("test-changefeed") @@ -210,13 +208,13 @@ func (s *managerSuite) TestClose(c *check.C) { }) s.tester.MustApplyPatches() _, err = s.manager.Tick(ctx, s.state) - c.Assert(err, check.IsNil) + require.Nil(t, err) s.tester.MustApplyPatches() - c.Assert(s.manager.processors, check.HasLen, 1) + require.Len(t, s.manager.processors, 1) s.manager.AsyncClose() _, err = s.manager.Tick(ctx, s.state) - c.Assert(cerrors.ErrReactorFinished.Equal(errors.Cause(err)), check.IsTrue) + require.True(t, cerrors.ErrReactorFinished.Equal(errors.Cause(err))) s.tester.MustApplyPatches() - c.Assert(s.manager.processors, check.HasLen, 0) + require.Len(t, s.manager.processors, 0) } diff --git a/cdc/processor/pipeline/sink.go b/cdc/processor/pipeline/sink.go index a0f72e7b68a..914d92bb6f5 100755 --- a/cdc/processor/pipeline/sink.go +++ b/cdc/processor/pipeline/sink.go @@ -75,8 +75,7 @@ type sinkNode struct { targetTs model.Ts barrierTs model.Ts - eventBuffer []*model.PolymorphicEvent - rowBuffer []*model.RowChangedEvent + rowBuffer []*model.RowChangedEvent flowController tableFlowController @@ -128,6 +127,8 @@ func (n *sinkNode) stop(ctx context.Context) (err error) { return } +// flushSink emits all rows in rowBuffer to the backend sink and flushes +// the backend sink. func (n *sinkNode) flushSink(ctx context.Context, resolvedTs model.Ts) (err error) { defer func() { if err != nil { @@ -147,7 +148,7 @@ func (n *sinkNode) flushSink(ctx context.Context, resolvedTs model.Ts) (err erro if resolvedTs <= n.checkpointTs { return nil } - if err := n.emitRow2Sink(ctx); err != nil { + if err := n.emitRowToSink(ctx); err != nil { return errors.Trace(err) } checkpointTs, err := n.sink.FlushRowChangedEvents(ctx, n.tableID, resolvedTs) @@ -172,7 +173,8 @@ func (n *sinkNode) flushSink(ctx context.Context, resolvedTs model.Ts) (err erro return nil } -func (n *sinkNode) emitEvent(ctx context.Context, event *model.PolymorphicEvent) error { +// addRowToBuffer checks event and adds event.Row to rowBuffer. +func (n *sinkNode) addRowToBuffer(ctx context.Context, event *model.PolymorphicEvent) error { if event == nil || event.Row == nil { log.Warn("skip emit nil event", zap.Any("event", event)) return nil @@ -182,7 +184,7 @@ func (n *sinkNode) emitEvent(ctx context.Context, event *model.PolymorphicEvent) preColLen := len(event.Row.PreColumns) // Some transactions could generate empty row change event, such as // begin; insert into t (id) values (1); delete from t where id=1; commit; - // Just ignore these row changed events + // Just ignore these row changed events. if colLen == 0 && preColLen == 0 { log.Warn("skip emit empty row event", zap.Any("event", event)) return nil @@ -198,18 +200,18 @@ func (n *sinkNode) emitEvent(ctx context.Context, event *model.PolymorphicEvent) return errors.Trace(err) } // NOTICE: Please do not change the order, the delete event always comes before the insert event. - n.eventBuffer = append(n.eventBuffer, deleteEvent, insertEvent) + n.rowBuffer = append(n.rowBuffer, deleteEvent.Row, insertEvent.Row) } else { // If the handle key columns are not updated, PreColumns is directly ignored. event.Row.PreColumns = nil - n.eventBuffer = append(n.eventBuffer, event) + n.rowBuffer = append(n.rowBuffer, event.Row) } } else { - n.eventBuffer = append(n.eventBuffer, event) + n.rowBuffer = append(n.rowBuffer, event.Row) } - if len(n.eventBuffer) >= defaultSyncResolvedBatch { - if err := n.emitRow2Sink(ctx); err != nil { + if len(n.rowBuffer) >= defaultSyncResolvedBatch { + if err := n.emitRowToSink(ctx); err != nil { return errors.Trace(err) } } @@ -280,7 +282,7 @@ func splitUpdateEvent(updateEvent *model.PolymorphicEvent) (*model.PolymorphicEv return &deleteEvent, &insertEvent, nil } -// clear event buffer and row buffer. +// clearBuffers clears rowBuffer. // Also, it dereferences data that are held by buffers. func (n *sinkNode) clearBuffers() { // Do not hog memory. @@ -292,21 +294,10 @@ func (n *sinkNode) clearBuffers() { } n.rowBuffer = n.rowBuffer[:0] } - - if cap(n.eventBuffer) > defaultSyncResolvedBatch { - n.eventBuffer = make([]*model.PolymorphicEvent, 0, defaultSyncResolvedBatch) - } else { - for i := range n.eventBuffer { - n.eventBuffer[i] = nil - } - n.eventBuffer = n.eventBuffer[:0] - } } -func (n *sinkNode) emitRow2Sink(ctx context.Context) error { - for _, ev := range n.eventBuffer { - n.rowBuffer = append(n.rowBuffer, ev.Row) - } +// emitRowToSink emits the rows in rowBuffer to backend sink. +func (n *sinkNode) emitRowToSink(ctx context.Context) error { failpoint.Inject("ProcessorSyncResolvedPreEmit", func() { log.Info("Prepare to panic for ProcessorSyncResolvedPreEmit") time.Sleep(10 * time.Second) @@ -346,7 +337,7 @@ func (n *sinkNode) HandleMessage(ctx context.Context, msg pipeline.Message) (boo atomic.StoreUint64(&n.resolvedTs, msg.PolymorphicEvent.CRTs) return true, nil } - if err := n.emitEvent(ctx, event); err != nil { + if err := n.addRowToBuffer(ctx, event); err != nil { return false, errors.Trace(err) } case pipeline.MessageTypeTick: diff --git a/cdc/processor/pipeline/sink_test.go b/cdc/processor/pipeline/sink_test.go index b4227fe392c..172c14dc0c8 100644 --- a/cdc/processor/pipeline/sink_test.go +++ b/cdc/processor/pipeline/sink_test.go @@ -372,7 +372,7 @@ func TestIgnoreEmptyRowChangeEvent(t *testing.T) { // empty row, no Columns and PreColumns. require.Nil(t, node.Receive(pipeline.MockNodeContext4Test(ctx, pipeline.PolymorphicEventMessage(&model.PolymorphicEvent{CRTs: 1, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}, Row: &model.RowChangedEvent{CommitTs: 1}}), nil))) - require.Equal(t, 0, len(node.eventBuffer)) + require.Equal(t, 0, len(node.rowBuffer)) } func TestSplitUpdateEventWhenEnableOldValue(t *testing.T) { @@ -391,7 +391,7 @@ func TestSplitUpdateEventWhenEnableOldValue(t *testing.T) { // nil row. require.Nil(t, node.Receive(pipeline.MockNodeContext4Test(ctx, pipeline.PolymorphicEventMessage(&model.PolymorphicEvent{CRTs: 1, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}), nil))) - require.Equal(t, 0, len(node.eventBuffer)) + require.Equal(t, 0, len(node.rowBuffer)) columns := []*model.Column{ { @@ -424,9 +424,9 @@ func TestSplitUpdateEventWhenEnableOldValue(t *testing.T) { RawKV: &model.RawKVEntry{OpType: model.OpTypePut}, Row: &model.RowChangedEvent{CommitTs: 1, Columns: columns, PreColumns: preColumns}, }), nil))) - require.Equal(t, 1, len(node.eventBuffer)) - require.Equal(t, 2, len(node.eventBuffer[0].Row.Columns)) - require.Equal(t, 2, len(node.eventBuffer[0].Row.PreColumns)) + require.Equal(t, 1, len(node.rowBuffer)) + require.Equal(t, 2, len(node.rowBuffer[0].Columns)) + require.Equal(t, 2, len(node.rowBuffer[0].PreColumns)) } func TestSplitUpdateEventWhenDisableOldValue(t *testing.T) { @@ -447,7 +447,7 @@ func TestSplitUpdateEventWhenDisableOldValue(t *testing.T) { // nil row. require.Nil(t, node.Receive(pipeline.MockNodeContext4Test(ctx, pipeline.PolymorphicEventMessage(&model.PolymorphicEvent{CRTs: 1, RawKV: &model.RawKVEntry{OpType: model.OpTypePut}}), nil))) - require.Equal(t, 0, len(node.eventBuffer)) + require.Equal(t, 0, len(node.rowBuffer)) // No update to the handle key column. columns := []*model.Column{ @@ -482,12 +482,12 @@ func TestSplitUpdateEventWhenDisableOldValue(t *testing.T) { RawKV: &model.RawKVEntry{OpType: model.OpTypePut}, Row: &model.RowChangedEvent{CommitTs: 1, Columns: columns, PreColumns: preColumns}, }), nil))) - require.Equal(t, 1, len(node.eventBuffer)) - require.Equal(t, 2, len(node.eventBuffer[0].Row.Columns)) - require.Equal(t, 0, len(node.eventBuffer[0].Row.PreColumns)) + require.Equal(t, 1, len(node.rowBuffer)) + require.Equal(t, 2, len(node.rowBuffer[0].Columns)) + require.Equal(t, 0, len(node.rowBuffer[0].PreColumns)) // Cleanup. - node.eventBuffer = []*model.PolymorphicEvent{} + node.rowBuffer = []*model.RowChangedEvent{} // Update to the handle key column. columns = []*model.Column{ { @@ -522,21 +522,21 @@ func TestSplitUpdateEventWhenDisableOldValue(t *testing.T) { Row: &model.RowChangedEvent{CommitTs: 1, Columns: columns, PreColumns: preColumns}, }), nil))) // Split an update event into a delete and an insert event. - require.Equal(t, 2, len(node.eventBuffer)) + require.Equal(t, 2, len(node.rowBuffer)) deleteEventIndex := 0 - require.Equal(t, 0, len(node.eventBuffer[deleteEventIndex].Row.Columns)) - require.Equal(t, 2, len(node.eventBuffer[deleteEventIndex].Row.PreColumns)) + require.Equal(t, 0, len(node.rowBuffer[deleteEventIndex].Columns)) + require.Equal(t, 2, len(node.rowBuffer[deleteEventIndex].PreColumns)) nonHandleKeyColIndex := 0 handleKeyColIndex := 1 // NOTICE: When old value disabled, we only keep the handle key pre cols. - require.Nil(t, node.eventBuffer[deleteEventIndex].Row.PreColumns[nonHandleKeyColIndex]) - require.Equal(t, "col2", node.eventBuffer[deleteEventIndex].Row.PreColumns[handleKeyColIndex].Name) - require.True(t, node.eventBuffer[deleteEventIndex].Row.PreColumns[handleKeyColIndex].Flag.IsHandleKey()) + require.Nil(t, node.rowBuffer[deleteEventIndex].PreColumns[nonHandleKeyColIndex]) + require.Equal(t, "col2", node.rowBuffer[deleteEventIndex].PreColumns[handleKeyColIndex].Name) + require.True(t, node.rowBuffer[deleteEventIndex].PreColumns[handleKeyColIndex].Flag.IsHandleKey()) insertEventIndex := 1 - require.Equal(t, 2, len(node.eventBuffer[insertEventIndex].Row.Columns)) - require.Equal(t, 0, len(node.eventBuffer[insertEventIndex].Row.PreColumns)) + require.Equal(t, 2, len(node.rowBuffer[insertEventIndex].Columns)) + require.Equal(t, 0, len(node.rowBuffer[insertEventIndex].PreColumns)) } type flushFlowController struct { diff --git a/cdc/processor/pipeline/sorter.go b/cdc/processor/pipeline/sorter.go index 90ecad4b672..0368d27a758 100644 --- a/cdc/processor/pipeline/sorter.go +++ b/cdc/processor/pipeline/sorter.go @@ -140,14 +140,6 @@ func (n *sorterNode) StartActorNode(ctx pipeline.NodeContext, isTableActorMode b ctx.Throw(errors.Trace(eventSorter.Run(stdCtx))) return nil }) - n.eg.Go(func() error { - // Since the flowController is implemented by `Cond`, it is not cancelable - // by a context. We need to listen on cancellation and aborts the flowController - // manually. - <-stdCtx.Done() - n.flowController.Abort() - return nil - }) n.eg.Go(func() error { lastSentResolvedTs := uint64(0) lastSendResolvedTsTime := time.Now() // the time at which we last sent a resolved-ts. @@ -176,10 +168,9 @@ func (n *sorterNode) StartActorNode(ctx pipeline.NodeContext, isTableActorMode b // this separate goroutine to prevent blocking // the whole pipeline. msg.SetUpFinishedChan() - select { - case <-ctx.Done(): - return nil - case n.mounter.Input() <- msg: + err := n.mounter.AddEntry(ctx, msg) + if err != nil { + return errors.Trace(err) } commitTs := msg.CRTs @@ -198,7 +189,7 @@ func (n *sorterNode) StartActorNode(ctx pipeline.NodeContext, isTableActorMode b } // Must wait before accessing msg.Row - err := msg.WaitPrepare(ctx) + err = msg.WaitPrepare(ctx) if err != nil { if errors.Cause(err) != context.Canceled { ctx.Throw(err) @@ -315,6 +306,10 @@ func (n *sorterNode) Destroy(ctx pipeline.NodeContext) error { log.Warn("schedule table cleanup task failed", zap.Error(err)) } } + // Since the flowController is implemented by `Cond`, it is not cancelable by a context + // the flowController will be blocked in a background goroutine, + // We need to abort the flowController manually in the nodeRunner + n.flowController.Abort() return n.eg.Wait() } diff --git a/cdc/processor/pipeline/table.go b/cdc/processor/pipeline/table.go index b0a45e269d5..725eeba2bb8 100644 --- a/cdc/processor/pipeline/table.go +++ b/cdc/processor/pipeline/table.go @@ -65,7 +65,7 @@ type tablePipelineImpl struct { tableID int64 markTableID int64 - tableName string // quoted schema and table, used in metircs only + tableName string // quoted schema and table, used in metrics only sorterNode *sorterNode sinkNode *sinkNode @@ -108,7 +108,7 @@ func (t *tablePipelineImpl) UpdateBarrierTs(ts model.Ts) { } } -// AsyncStop tells the pipeline to stop, and returns true is the pipeline is already stopped. +// AsyncStop tells the pipeline to stop, and returns true if the pipeline is already stopped. func (t *tablePipelineImpl) AsyncStop(targetTs model.Ts) bool { err := t.p.SendToFirstNode(pipeline.CommandMessage(&pipeline.Command{ Tp: pipeline.CommandTypeStop, diff --git a/cdc/processor/processor.go b/cdc/processor/processor.go index e4f9251acd3..b66f7831089 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -363,6 +363,7 @@ func (p *processor) tick(ctx cdcContext.Context, state *orchestrator.ChangefeedR p.handleWorkload() } p.doGCSchemaStorage(ctx) + p.metricSyncTableNumGauge.Set(float64(len(p.tables))) if p.newSchedulerEnabled { if err := p.agent.Tick(ctx); err != nil { @@ -974,17 +975,6 @@ func (p *processor) createTablePipelineImpl(ctx cdcContext.Context, tableID mode sink, p.changefeed.Info.GetTargetTs(), ) - p.wg.Add(1) - p.metricSyncTableNumGauge.Inc() - go func() { - table.Wait() - p.wg.Done() - p.metricSyncTableNumGauge.Dec() - log.Debug("Table pipeline exited", zap.Int64("tableID", tableID), - cdcContext.ZapFieldChangefeed(ctx), - zap.String("name", table.Name()), - zap.Any("replicaInfo", replicaInfo)) - }() if p.redoManager.Enabled() { p.redoManager.AddTable(tableID, replicaInfo.StartTs) diff --git a/cdc/processor/processor_test.go b/cdc/processor/processor_test.go index b171473d167..611befc491c 100644 --- a/cdc/processor/processor_test.go +++ b/cdc/processor/processor_test.go @@ -21,7 +21,6 @@ import ( "sync/atomic" "testing" - "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/entry" @@ -34,21 +33,15 @@ import ( cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/orchestrator" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) -func Test(t *testing.T) { check.TestingT(t) } - -type processorSuite struct{} - -var _ = check.Suite(&processorSuite{}) - // processor needs to implement TableExecutor. var _ scheduler.TableExecutor = (*processor)(nil) func newProcessor4Test( ctx cdcContext.Context, - c *check.C, + t *testing.T, createTablePipeline func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error), ) *processor { p := newProcessor(ctx) @@ -59,12 +52,12 @@ func newProcessor4Test( p.sinkManager = &sink.Manager{} p.redoManager = redo.NewDisabledManager() p.createTablePipeline = createTablePipeline - p.schemaStorage = &mockSchemaStorage{c: c, resolvedTs: math.MaxUint64} + p.schemaStorage = &mockSchemaStorage{t: t, resolvedTs: math.MaxUint64} return p } -func initProcessor4Test(ctx cdcContext.Context, c *check.C) (*processor, *orchestrator.ReactorStateTester) { - p := newProcessor4Test(ctx, c, func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) { +func initProcessor4Test(ctx cdcContext.Context, t *testing.T) (*processor, *orchestrator.ReactorStateTester) { + p := newProcessor4Test(ctx, t, func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) { return &mockTablePipeline{ tableID: tableID, name: fmt.Sprintf("`test`.`table%d`", tableID), @@ -74,7 +67,7 @@ func initProcessor4Test(ctx cdcContext.Context, c *check.C) (*processor, *orches }, nil }) p.changefeed = orchestrator.NewChangefeedReactorState(ctx.ChangefeedVars().ID) - return p, orchestrator.NewReactorStateTester(c, p.changefeed, map[string]string{ + return p, orchestrator.NewReactorStateTester(t, p.changefeed, map[string]string{ "/tidb/cdc/capture/" + ctx.GlobalVars().CaptureInfo.ID: `{"id":"` + ctx.GlobalVars().CaptureInfo.ID + `","address":"127.0.0.1:8300"}`, "/tidb/cdc/changefeed/info/" + ctx.ChangefeedVars().ID: `{"sink-uri":"blackhole://","opts":{},"create-time":"2020-02-02T00:00:00.000000+00:00","start-ts":0,"target-ts":0,"admin-job-type":0,"sort-engine":"memory","sort-dir":".","config":{"case-sensitive":true,"enable-old-value":false,"force-replicate":false,"check-gc-safe-point":true,"filter":{"rules":["*.*"],"ignore-txn-start-ts":null,"ddl-allow-list":null},"mounter":{"worker-num":16},"sink":{"dispatchers":null,"protocol":"open-protocol"},"cyclic-replication":{"enable":false,"replica-id":0,"filter-replica-ids":null,"id-buckets":0,"sync-ddl":false},"scheduler":{"type":"table-number","polling-time":-1}},"state":"normal","history":null,"error":null,"sync-point-enabled":false,"sync-point-interval":600000000000}`, "/tidb/cdc/job/" + ctx.ChangefeedVars().ID: `{"resolved-ts":0,"checkpoint-ts":0,"admin-job-type":0}`, @@ -142,7 +135,7 @@ type mockSchemaStorage struct { // as we only need ResolvedTs() and DoGC() in unit tests. entry.SchemaStorage - c *check.C + t *testing.T lastGcTs uint64 resolvedTs uint64 } @@ -152,7 +145,7 @@ func (s *mockSchemaStorage) ResolvedTs() uint64 { } func (s *mockSchemaStorage) DoGC(ts uint64) uint64 { - s.c.Assert(s.lastGcTs, check.LessEqual, ts) + require.LessOrEqual(s.t, s.lastGcTs, ts) atomic.StoreUint64(&s.lastGcTs, ts) return ts } @@ -183,15 +176,14 @@ func (a *mockAgent) Close() error { return nil } -func (s *processorSuite) TestCheckTablesNum(c *check.C) { - defer testleak.AfterTest(c)() +func TestCheckTablesNum(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - p, tester := initProcessor4Test(ctx, c) + p, tester := initProcessor4Test(ctx, t) var err error _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID], &model.TaskPosition{ CheckPointTs: 0, ResolvedTs: 0, @@ -199,13 +191,13 @@ func (s *processorSuite) TestCheckTablesNum(c *check.C) { Error: nil, }) - p, tester = initProcessor4Test(ctx, c) + p, tester = initProcessor4Test(ctx, t) p.changefeed.Info.StartTs = 66 p.changefeed.Status.CheckpointTs = 88 _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID], &model.TaskPosition{ CheckPointTs: 88, ResolvedTs: 88, @@ -214,14 +206,13 @@ func (s *processorSuite) TestCheckTablesNum(c *check.C) { }) } -func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { - defer testleak.AfterTest(c)() +func TestHandleTableOperation4SingleTable(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - p, tester := initProcessor4Test(ctx, c) + p, tester := initProcessor4Test(ctx, t) var err error // init tick _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() p.changefeed.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { status.CheckpointTs = 90 @@ -236,7 +227,7 @@ func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { // no operation _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // add table, in processing @@ -247,9 +238,9 @@ func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { }) tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{ 66: {StartTs: 60}, }, @@ -260,9 +251,9 @@ func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { // add table, not finished _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{ 66: {StartTs: 60}, }, @@ -275,9 +266,9 @@ func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { table66 := p.tables[66].(*mockTablePipeline) table66.resolvedTs = 101 _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{ 66: {StartTs: 60}, }, @@ -285,13 +276,13 @@ func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { 66: {Delete: false, BoundaryTs: 60, Status: model.OperProcessed}, }, }) - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID].ResolvedTs, check.Equals, uint64(101)) + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID].ResolvedTs, uint64(101)) // finish the operation _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{ 66: {StartTs: 60}, }, @@ -310,21 +301,21 @@ func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { }) tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{}, Operation: map[int64]*model.TableOperation{ 66: {Delete: true, BoundaryTs: 120, Status: model.OperProcessed}, }, }) - c.Assert(table66.stopTs, check.Equals, uint64(120)) + require.Equal(t, table66.stopTs, uint64(120)) // remove table, not finished _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{}, Operation: map[int64]*model.TableOperation{ 66: {Delete: true, BoundaryTs: 120, Status: model.OperProcessed}, @@ -335,26 +326,25 @@ func (s *processorSuite) TestHandleTableOperation4SingleTable(c *check.C) { table66.status = tablepipeline.TableStatusStopped table66.checkpointTs = 121 _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{}, Operation: map[int64]*model.TableOperation{ 66: {Delete: true, BoundaryTs: 121, Status: model.OperFinished}, }, }) - c.Assert(table66.canceled, check.IsTrue) - c.Assert(p.tables[66], check.IsNil) + require.True(t, table66.canceled) + require.Nil(t, p.tables[66]) } -func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { - defer testleak.AfterTest(c)() +func TestHandleTableOperation4MultiTable(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - p, tester := initProcessor4Test(ctx, c) + p, tester := initProcessor4Test(ctx, t) var err error // init tick _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() p.changefeed.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { status.CheckpointTs = 20 @@ -370,7 +360,7 @@ func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { // no operation _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // add table, in processing @@ -384,9 +374,9 @@ func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { }) tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{ 1: {StartTs: 60}, 2: {StartTs: 50}, @@ -399,9 +389,9 @@ func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { 3: {Delete: false, BoundaryTs: 40, Status: model.OperProcessed}, }, }) - c.Assert(p.tables, check.HasLen, 4) - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID].CheckPointTs, check.Equals, uint64(30)) - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID].ResolvedTs, check.Equals, uint64(30)) + require.Len(t, p.tables, 4) + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID].CheckPointTs, uint64(30)) + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID].ResolvedTs, uint64(30)) // add table, push the resolvedTs, finished add table table1 := p.tables[1].(*mockTablePipeline) @@ -419,9 +409,9 @@ func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { }) tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{ 1: {StartTs: 60}, 2: {StartTs: 50}, @@ -433,18 +423,18 @@ func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { 3: {Delete: true, BoundaryTs: 60, Status: model.OperProcessed}, }, }) - c.Assert(p.tables, check.HasLen, 4) - c.Assert(table3.canceled, check.IsFalse) - c.Assert(table3.stopTs, check.Equals, uint64(60)) - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID].ResolvedTs, check.Equals, uint64(101)) + require.Len(t, p.tables, 4) + require.False(t, table3.canceled) + require.Equal(t, table3.stopTs, uint64(60)) + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID].ResolvedTs, uint64(101)) // finish remove operations table3.status = tablepipeline.TableStatusStopped table3.checkpointTs = 65 _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{ 1: {StartTs: 60}, 2: {StartTs: 50}, @@ -456,8 +446,8 @@ func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { 3: {Delete: true, BoundaryTs: 65, Status: model.OperFinished}, }, }) - c.Assert(p.tables, check.HasLen, 3) - c.Assert(table3.canceled, check.IsTrue) + require.Len(t, p.tables, 3) + require.True(t, table3.canceled) // clear finished operations cleanUpFinishedOpOperation(p.changefeed, p.captureInfo.ID, tester) @@ -471,25 +461,25 @@ func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { }) tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{}, Operation: map[int64]*model.TableOperation{ 1: {Delete: true, BoundaryTs: 120, Status: model.OperProcessed}, 4: {Delete: true, BoundaryTs: 120, Status: model.OperProcessed}, }, }) - c.Assert(table1.stopTs, check.Equals, uint64(120)) - c.Assert(table4.stopTs, check.Equals, uint64(120)) - c.Assert(table2.canceled, check.IsTrue) - c.Assert(p.tables, check.HasLen, 2) + require.Equal(t, table1.stopTs, uint64(120)) + require.Equal(t, table4.stopTs, uint64(120)) + require.True(t, table2.canceled) + require.Len(t, p.tables, 2) // remove table, not finished _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{}, Operation: map[int64]*model.TableOperation{ 1: {Delete: true, BoundaryTs: 120, Status: model.OperProcessed}, @@ -503,24 +493,23 @@ func (s *processorSuite) TestHandleTableOperation4MultiTable(c *check.C) { table4.status = tablepipeline.TableStatusStopped table4.checkpointTs = 122 _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{}, Operation: map[int64]*model.TableOperation{ 1: {Delete: true, BoundaryTs: 121, Status: model.OperFinished}, 4: {Delete: true, BoundaryTs: 122, Status: model.OperFinished}, }, }) - c.Assert(table1.canceled, check.IsTrue) - c.Assert(table4.canceled, check.IsTrue) - c.Assert(p.tables, check.HasLen, 0) + require.True(t, table1.canceled) + require.True(t, table4.canceled) + require.Len(t, p.tables, 0) } -func (s *processorSuite) TestTableExecutor(c *check.C) { - defer testleak.AfterTest(c)() +func TestTableExecutor(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - p, tester := initProcessor4Test(ctx, c) + p, tester := initProcessor4Test(ctx, t) p.newSchedulerEnabled = true p.lazyInit = func(ctx cdcContext.Context) error { p.agent = &mockAgent{executor: p} @@ -530,7 +519,7 @@ func (s *processorSuite) TestTableExecutor(c *check.C) { var err error // init tick _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() p.changefeed.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { status.CheckpointTs = 20 @@ -546,40 +535,38 @@ func (s *processorSuite) TestTableExecutor(c *check.C) { // no operation _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() ok, err := p.AddTable(ctx, 1) - c.Check(err, check.IsNil) - c.Check(ok, check.IsTrue) + require.Nil(t, err) + require.True(t, ok) ok, err = p.AddTable(ctx, 2) - c.Check(err, check.IsNil) - c.Check(ok, check.IsTrue) + require.Nil(t, err) + require.True(t, ok) ok, err = p.AddTable(ctx, 3) - c.Check(err, check.IsNil) - c.Check(ok, check.IsTrue) + require.Nil(t, err) + require.True(t, ok) ok, err = p.AddTable(ctx, 4) - c.Check(err, check.IsNil) - c.Check(ok, check.IsTrue) - - c.Assert(p.tables, check.HasLen, 4) + require.Nil(t, err) + require.True(t, ok) + require.Len(t, p.tables, 4) checkpointTs := p.agent.GetLastSentCheckpointTs() - c.Assert(checkpointTs, check.Equals, uint64(0)) + require.Equal(t, checkpointTs, uint64(0)) done := p.IsAddTableFinished(ctx, 1) - c.Check(done, check.IsFalse) + require.False(t, done) done = p.IsAddTableFinished(ctx, 2) - c.Check(done, check.IsFalse) + require.False(t, done) done = p.IsAddTableFinished(ctx, 3) - c.Check(done, check.IsFalse) + require.False(t, done) done = p.IsAddTableFinished(ctx, 4) - c.Check(done, check.IsFalse) - - c.Assert(p.tables, check.HasLen, 4) + require.False(t, done) + require.Len(t, p.tables, 4) _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // add table, push the resolvedTs, finished add table @@ -598,16 +585,16 @@ func (s *processorSuite) TestTableExecutor(c *check.C) { table4.checkpointTs = 30 done = p.IsAddTableFinished(ctx, 1) - c.Check(done, check.IsTrue) + require.True(t, done) done = p.IsAddTableFinished(ctx, 2) - c.Check(done, check.IsTrue) + require.True(t, done) done = p.IsAddTableFinished(ctx, 3) - c.Check(done, check.IsTrue) + require.True(t, done) done = p.IsAddTableFinished(ctx, 4) - c.Check(done, check.IsTrue) + require.True(t, done) _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() table1.checkpointTs = 75 @@ -616,77 +603,78 @@ func (s *processorSuite) TestTableExecutor(c *check.C) { table4.checkpointTs = 75 _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() checkpointTs = p.agent.GetLastSentCheckpointTs() - c.Assert(checkpointTs, check.Equals, uint64(60)) + require.Equal(t, checkpointTs, uint64(60)) - updateChangeFeedPosition(c, tester, ctx.ChangefeedVars().ID, 103, 60) + updateChangeFeedPosition(t, tester, ctx.ChangefeedVars().ID, 103, 60) _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() ok, err = p.RemoveTable(ctx, 3) - c.Check(err, check.IsNil) - c.Check(ok, check.IsTrue) + require.Nil(t, err) + require.True(t, ok) _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) + tester.MustApplyPatches() - c.Assert(p.tables, check.HasLen, 4) - c.Assert(table3.canceled, check.IsFalse) - c.Assert(table3.stopTs, check.Equals, uint64(60)) + require.Len(t, p.tables, 4) + require.False(t, table3.canceled) + require.Equal(t, table3.stopTs, uint64(60)) done = p.IsRemoveTableFinished(ctx, 3) - c.Assert(done, check.IsFalse) + require.False(t, done) _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() checkpointTs = p.agent.GetLastSentCheckpointTs() - c.Assert(checkpointTs, check.Equals, uint64(60)) + require.Equal(t, checkpointTs, uint64(60)) // finish remove operations table3.status = tablepipeline.TableStatusStopped table3.checkpointTs = 65 _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) + tester.MustApplyPatches() - c.Assert(p.tables, check.HasLen, 4) - c.Assert(table3.canceled, check.IsFalse) + require.Len(t, p.tables, 4) + require.False(t, table3.canceled) done = p.IsRemoveTableFinished(ctx, 3) - c.Assert(done, check.IsTrue) + require.True(t, done) - c.Assert(p.tables, check.HasLen, 3) - c.Assert(table3.canceled, check.IsTrue) + require.Len(t, p.tables, 3) + require.True(t, table3.canceled) _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() checkpointTs = p.agent.GetLastSentCheckpointTs() - c.Assert(checkpointTs, check.Equals, uint64(75)) + require.Equal(t, checkpointTs, uint64(75)) err = p.Close() - c.Assert(err, check.IsNil) - c.Assert(p.agent, check.IsNil) + require.Nil(t, err) + require.Nil(t, p.agent) } -func (s *processorSuite) TestInitTable(c *check.C) { - defer testleak.AfterTest(c)() +func TestInitTable(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - p, tester := initProcessor4Test(ctx, c) + p, tester := initProcessor4Test(ctx, t) var err error // init tick _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() p.changefeed.PatchTaskStatus(p.captureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { @@ -696,28 +684,27 @@ func (s *processorSuite) TestInitTable(c *check.C) { }) tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.tables[1], check.Not(check.IsNil)) - c.Assert(p.tables[2], check.Not(check.IsNil)) + require.NotNil(t, p.tables[1]) + require.NotNil(t, p.tables[2]) } -func (s *processorSuite) TestProcessorError(c *check.C) { - defer testleak.AfterTest(c)() +func TestProcessorError(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - p, tester := initProcessor4Test(ctx, c) + p, tester := initProcessor4Test(ctx, t) var err error // init tick _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // send a abnormal error p.sendError(cerror.ErrSinkURIInvalid) _, err = p.Tick(ctx, p.changefeed) tester.MustApplyPatches() - c.Assert(cerror.ErrReactorFinished.Equal(errors.Cause(err)), check.IsTrue) - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, &model.TaskPosition{ + require.True(t, cerror.ErrReactorFinished.Equal(errors.Cause(err))) + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID], &model.TaskPosition{ Error: &model.RunningError{ Addr: "127.0.0.1:0000", Code: "CDC:ErrSinkURIInvalid", @@ -725,30 +712,29 @@ func (s *processorSuite) TestProcessorError(c *check.C) { }, }) - p, tester = initProcessor4Test(ctx, c) + p, tester = initProcessor4Test(ctx, t) // init tick _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // send a normal error p.sendError(context.Canceled) _, err = p.Tick(ctx, p.changefeed) tester.MustApplyPatches() - c.Assert(cerror.ErrReactorFinished.Equal(errors.Cause(err)), check.IsTrue) - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, &model.TaskPosition{ + require.True(t, cerror.ErrReactorFinished.Equal(errors.Cause(err))) + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID], &model.TaskPosition{ Error: nil, }) } -func (s *processorSuite) TestProcessorExit(c *check.C) { - defer testleak.AfterTest(c)() +func TestProcessorExit(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - p, tester := initProcessor4Test(ctx, c) + p, tester := initProcessor4Test(ctx, t) var err error // init tick _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // stop the changefeed @@ -762,21 +748,20 @@ func (s *processorSuite) TestProcessorExit(c *check.C) { }) tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) - c.Assert(cerror.ErrReactorFinished.Equal(errors.Cause(err)), check.IsTrue) + require.True(t, cerror.ErrReactorFinished.Equal(errors.Cause(err))) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, &model.TaskPosition{ + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID], &model.TaskPosition{ Error: nil, }) } -func (s *processorSuite) TestProcessorClose(c *check.C) { - defer testleak.AfterTest(c)() +func TestProcessorClose(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - p, tester := initProcessor4Test(ctx, c) + p, tester := initProcessor4Test(ctx, t) var err error // init tick _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // add tables @@ -787,7 +772,7 @@ func (s *processorSuite) TestProcessorClose(c *check.C) { }) tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // push the resolvedTs and checkpointTs @@ -801,27 +786,27 @@ func (s *processorSuite) TestProcessorClose(c *check.C) { p.tables[1].(*mockTablePipeline).checkpointTs = 90 p.tables[2].(*mockTablePipeline).checkpointTs = 95 _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, &model.TaskPosition{ + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID], &model.TaskPosition{ CheckPointTs: 90, ResolvedTs: 90, Error: nil, }) - c.Assert(p.changefeed.TaskStatuses[p.captureInfo.ID], check.DeepEquals, &model.TaskStatus{ + require.Equal(t, p.changefeed.TaskStatuses[p.captureInfo.ID], &model.TaskStatus{ Tables: map[int64]*model.TableReplicaInfo{1: {StartTs: 20}, 2: {StartTs: 30}}, }) - c.Assert(p.changefeed.Workloads[p.captureInfo.ID], check.DeepEquals, model.TaskWorkload{1: {Workload: 1}, 2: {Workload: 1}}) + require.Equal(t, p.changefeed.Workloads[p.captureInfo.ID], model.TaskWorkload{1: {Workload: 1}, 2: {Workload: 1}}) - c.Assert(p.Close(), check.IsNil) + require.Nil(t, p.Close()) tester.MustApplyPatches() - c.Assert(p.tables[1].(*mockTablePipeline).canceled, check.IsTrue) - c.Assert(p.tables[2].(*mockTablePipeline).canceled, check.IsTrue) + require.True(t, p.tables[1].(*mockTablePipeline).canceled) + require.True(t, p.tables[2].(*mockTablePipeline).canceled) - p, tester = initProcessor4Test(ctx, c) + p, tester = initProcessor4Test(ctx, t) // init tick _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // add tables @@ -832,30 +817,29 @@ func (s *processorSuite) TestProcessorClose(c *check.C) { }) tester.MustApplyPatches() _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // send error p.sendError(cerror.ErrSinkURIInvalid) _, err = p.Tick(ctx, p.changefeed) - c.Assert(cerror.ErrReactorFinished.Equal(errors.Cause(err)), check.IsTrue) + require.True(t, cerror.ErrReactorFinished.Equal(errors.Cause(err))) tester.MustApplyPatches() - c.Assert(p.Close(), check.IsNil) + require.Nil(t, p.Close()) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID].Error, check.DeepEquals, &model.RunningError{ + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID].Error, &model.RunningError{ Addr: "127.0.0.1:0000", Code: "CDC:ErrSinkURIInvalid", Message: "[CDC:ErrSinkURIInvalid]sink uri invalid", }) - c.Assert(p.tables[1].(*mockTablePipeline).canceled, check.IsTrue) - c.Assert(p.tables[2].(*mockTablePipeline).canceled, check.IsTrue) + require.True(t, p.tables[1].(*mockTablePipeline).canceled) + require.True(t, p.tables[2].(*mockTablePipeline).canceled) } -func (s *processorSuite) TestPositionDeleted(c *check.C) { - defer testleak.AfterTest(c)() +func TestPositionDeleted(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - p, tester := initProcessor4Test(ctx, c) + p, tester := initProcessor4Test(ctx, t) p.changefeed.PatchTaskStatus(p.captureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { status.Tables[1] = &model.TableReplicaInfo{StartTs: 30} status.Tables[2] = &model.TableReplicaInfo{StartTs: 40} @@ -864,14 +848,14 @@ func (s *processorSuite) TestPositionDeleted(c *check.C) { var err error // init tick _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // cal position _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, &model.TaskPosition{ + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID], &model.TaskPosition{ CheckPointTs: 30, ResolvedTs: 30, }) @@ -883,27 +867,26 @@ func (s *processorSuite) TestPositionDeleted(c *check.C) { tester.MustApplyPatches() // position created again _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, &model.TaskPosition{ + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID], &model.TaskPosition{ CheckPointTs: 0, ResolvedTs: 0, }) // cal position _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - c.Assert(p.changefeed.TaskPositions[p.captureInfo.ID], check.DeepEquals, &model.TaskPosition{ + require.Equal(t, p.changefeed.TaskPositions[p.captureInfo.ID], &model.TaskPosition{ CheckPointTs: 30, ResolvedTs: 30, }) } -func (s *processorSuite) TestSchemaGC(c *check.C) { - defer testleak.AfterTest(c)() +func TestSchemaGC(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - p, tester := initProcessor4Test(ctx, c) + p, tester := initProcessor4Test(ctx, t) p.changefeed.PatchTaskStatus(p.captureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { status.Tables[1] = &model.TableReplicaInfo{StartTs: 30} status.Tables[2] = &model.TableReplicaInfo{StartTs: 40} @@ -913,17 +896,17 @@ func (s *processorSuite) TestSchemaGC(c *check.C) { var err error // init tick _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() - updateChangeFeedPosition(c, tester, "changefeed-id-test", 50, 50) + updateChangeFeedPosition(t, tester, "changefeed-id-test", 50, 50) _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // GC Ts should be (checkpoint - 1). - c.Assert(p.schemaStorage.(*mockSchemaStorage).lastGcTs, check.Equals, uint64(49)) - c.Assert(p.lastSchemaTs, check.Equals, uint64(49)) + require.Equal(t, p.schemaStorage.(*mockSchemaStorage).lastGcTs, uint64(49)) + require.Equal(t, p.lastSchemaTs, uint64(49)) } func cleanUpFinishedOpOperation(state *orchestrator.ChangefeedReactorState, captureID model.CaptureID, tester *orchestrator.ReactorStateTester) { @@ -941,7 +924,7 @@ func cleanUpFinishedOpOperation(state *orchestrator.ChangefeedReactorState, capt tester.MustApplyPatches() } -func updateChangeFeedPosition(c *check.C, tester *orchestrator.ReactorStateTester, cfID model.ChangeFeedID, resolvedTs, checkpointTs model.Ts) { +func updateChangeFeedPosition(t *testing.T, tester *orchestrator.ReactorStateTester, cfID model.ChangeFeedID, resolvedTs, checkpointTs model.Ts) { key := etcd.CDCKey{ Tp: etcd.CDCKeyTypeChangeFeedStatus, ChangefeedID: cfID, @@ -953,14 +936,12 @@ func updateChangeFeedPosition(c *check.C, tester *orchestrator.ReactorStateTeste CheckpointTs: checkpointTs, } valueBytes, err := json.Marshal(cfStatus) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustUpdate(keyStr, valueBytes) } -func (s *processorSuite) TestIgnorableError(c *check.C) { - defer testleak.AfterTest(c)() - +func TestIgnorableError(t *testing.T) { testCases := []struct { err error ignorable bool @@ -974,14 +955,13 @@ func (s *processorSuite) TestIgnorableError(c *check.C) { {errors.New("test error"), false}, } for _, tc := range testCases { - c.Assert(isProcessorIgnorableError(tc.err), check.Equals, tc.ignorable) + require.Equal(t, isProcessorIgnorableError(tc.err), tc.ignorable) } } -func (s *processorSuite) TestUpdateBarrierTs(c *check.C) { - defer testleak.AfterTest(c)() +func TestUpdateBarrierTs(t *testing.T) { ctx := cdcContext.NewBackendContext4Test(true) - p, tester := initProcessor4Test(ctx, c) + p, tester := initProcessor4Test(ctx, t) p.changefeed.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { status.CheckpointTs = 5 status.ResolvedTs = 10 @@ -995,11 +975,11 @@ func (s *processorSuite) TestUpdateBarrierTs(c *check.C) { // init tick, add table OperDispatched. _, err := p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // tick again, add table OperProcessed. _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() // Global resolved ts has advanced while schema storage stalls. @@ -1008,16 +988,16 @@ func (s *processorSuite) TestUpdateBarrierTs(c *check.C) { return status, true, nil }) _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() tb := p.tables[model.TableID(1)].(*mockTablePipeline) - c.Assert(tb.barrierTs, check.Equals, uint64(10)) + require.Equal(t, tb.barrierTs, uint64(10)) // Schema storage has advanced too. p.schemaStorage.(*mockSchemaStorage).resolvedTs = 15 _, err = p.Tick(ctx, p.changefeed) - c.Assert(err, check.IsNil) + require.Nil(t, err) tester.MustApplyPatches() tb = p.tables[model.TableID(1)].(*mockTablePipeline) - c.Assert(tb.barrierTs, check.Equals, uint64(15)) + require.Equal(t, tb.barrierTs, uint64(15)) } diff --git a/cdc/puller/puller.go b/cdc/puller/puller.go index a04831a0eba..ed698dec057 100644 --- a/cdc/puller/puller.go +++ b/cdc/puller/puller.go @@ -139,21 +139,11 @@ func (p *pullerImpl) Run(ctx context.Context) error { txnCollectCounter.DeleteLabelValues(captureAddr, changefeedID, "kv") txnCollectCounter.DeleteLabelValues(captureAddr, changefeedID, "resolved") }() - g.Go(func() error { - for { - select { - case <-ctx.Done(): - return nil - case <-time.After(15 * time.Second): - metricEventChanSize.Observe(float64(len(eventCh))) - metricOutputChanSize.Observe(float64(len(p.outputCh))) - metricPullerResolvedTs.Set(float64(oracle.ExtractPhysical(atomic.LoadUint64(&p.resolvedTs)))) - } - } - }) lastResolvedTs := p.checkpointTs g.Go(func() error { + metricsTicker := time.NewTicker(15 * time.Second) + defer metricsTicker.Stop() output := func(raw *model.RawKVEntry) error { // even after https://github.com/pingcap/tiflow/pull/2038, kv client // could still miss region change notification, which leads to resolved @@ -181,6 +171,11 @@ func (p *pullerImpl) Run(ctx context.Context) error { for { var e model.RegionFeedEvent select { + case <-metricsTicker.C: + metricEventChanSize.Observe(float64(len(eventCh))) + metricOutputChanSize.Observe(float64(len(p.outputCh))) + metricPullerResolvedTs.Set(float64(oracle.ExtractPhysical(atomic.LoadUint64(&p.resolvedTs)))) + continue case e = <-eventCh: case <-ctx.Done(): return errors.Trace(ctx.Err()) diff --git a/cdc/redo/manager.go b/cdc/redo/manager.go index 011477eb9be..96b979dde41 100644 --- a/cdc/redo/manager.go +++ b/cdc/redo/manager.go @@ -54,13 +54,9 @@ const ( ) const ( - // supposing to replicate 10k tables, each table has one cached change averagely. - // approximate 156.25KB + // supposing to replicate 10k tables, each table has one cached changce averagely. logBufferChanSize = 10000 - // supposing to replicate 10k tables, each table has one resolvedTs change averagely. - // approximate 156.25KB - flushBufferChanSize = 10000 - logBufferTimeout = time.Minute * 10 + logBufferTimeout = time.Minute * 10 ) // IsValidConsistentLevel checks whether a give consistent level is valid @@ -127,12 +123,6 @@ type cacheRows struct { rows []*model.RowChangedEvent } -// resolvedEvent represents a resolvedTs event -type resolvedEvent struct { - tableID model.TableID - resolvedTs model.Ts -} - // ManagerImpl manages redo log writer, buffers un-persistent redo logs, calculates // redo log resolved ts. It implements LogManager interface. type ManagerImpl struct { @@ -140,9 +130,8 @@ type ManagerImpl struct { level ConsistentLevelType storageType consistentStorage - flushBuffer chan resolvedEvent - logBuffer chan cacheRows - writer writer.RedoLogWriter + logBuffer chan cacheRows + writer writer.RedoLogWriter minResolvedTs uint64 tableIDs []model.TableID @@ -168,8 +157,7 @@ func NewManager(ctx context.Context, cfg *config.ConsistentConfig, opts *Manager level: ConsistentLevelType(cfg.Level), storageType: consistentStorage(uri.Scheme), rtsMap: make(map[model.TableID]uint64), - logBuffer: make(chan cacheRows, logBufferChanSize), /* approximate 0.228MB */ - flushBuffer: make(chan resolvedEvent, flushBufferChanSize), /* approximate 0.152MB */ + logBuffer: make(chan cacheRows, logBufferChanSize), } switch m.storageType { @@ -208,7 +196,6 @@ func NewManager(ctx context.Context, cfg *config.ConsistentConfig, opts *Manager if opts.EnableBgRunner { go m.bgUpdateResolvedTs(ctx, opts.ErrCh) - go m.bgFlushLog(ctx, opts.ErrCh) go m.bgWriteLog(ctx, opts.ErrCh) } return m, nil @@ -291,16 +278,7 @@ func (m *ManagerImpl) FlushLog( return nil } defer atomic.StoreInt64(&m.flushing, 0) - - select { - case <-ctx.Done(): - return ctx.Err() - case m.flushBuffer <- resolvedEvent{ - tableID: tableID, - resolvedTs: resolvedTs, - }: - return nil - } + return m.writer.FlushLog(ctx, tableID, resolvedTs) } // EmitDDLEvent sends DDL event to redo log writer @@ -397,7 +375,7 @@ func (m *ManagerImpl) bgUpdateResolvedTs(ctx context.Context, errCh chan<- error select { case errCh <- err: default: - log.Error("redo log manager err channel is full", zap.Error(err)) + log.Error("err channel is full", zap.Error(err)) } return } @@ -420,30 +398,10 @@ func (m *ManagerImpl) bgWriteLog(ctx context.Context, errCh chan<- error) { select { case errCh <- err: default: - log.Error("redo log manager err channel is full", zap.Error(err)) - } - return - } - } - } -} - -func (m *ManagerImpl) bgFlushLog(ctx context.Context, errCh chan<- error) { - for { - select { - case <-ctx.Done(): - return - case event := <-m.flushBuffer: - err := m.writer.FlushLog(ctx, event.tableID, event.resolvedTs) - if err != nil { - select { - case errCh <- err: - default: - log.Error("redo log manager err channel is full", zap.Error(err)) + log.Error("err channel is full", zap.Error(err)) } return } - } } } diff --git a/cdc/server.go b/cdc/server.go index 86d8529fbfb..cf3ab6f651c 100644 --- a/cdc/server.go +++ b/cdc/server.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" tidbkv "github.com/pingcap/tidb/kv" - "github.com/pingcap/tiflow/cdc/api" "github.com/pingcap/tiflow/cdc/capture" "github.com/pingcap/tiflow/cdc/kv" "github.com/pingcap/tiflow/cdc/sorter/unified" @@ -218,16 +217,9 @@ func (s *Server) startStatusHTTP(lis net.Listener) error { // 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()) - // Register APIs. - api.RegisterRoutes(router, s.capture, registry) + RegisterRoutes(router, s.capture, registry) // No need to configure TLS because it is already handled by `s.tcpServer`. s.statusServer = &http.Server{Handler: router} diff --git a/cdc/server_test.go b/cdc/server_test.go index 567eb4c71f7..b6eccf3c74b 100644 --- a/cdc/server_test.go +++ b/cdc/server_test.go @@ -15,27 +15,36 @@ package cdc import ( "context" + "crypto/tls" + "encoding/json" + "fmt" + "net/http" "net/url" "os" "os/user" "path/filepath" "runtime" + "sync" "testing" "time" - "github.com/pingcap/check" + "github.com/pingcap/tidb/br/pkg/httputil" + "github.com/pingcap/tiflow/cdc/capture" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" + cerrors "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/retry" + security2 "github.com/pingcap/tiflow/pkg/security" "github.com/pingcap/tiflow/pkg/util" - "github.com/pingcap/tiflow/pkg/util/testleak" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/tempurl" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/embed" "golang.org/x/sync/errgroup" ) -type serverSuite struct { +type testServer struct { server *Server e *embed.Etcd clientURL *url.URL @@ -44,19 +53,20 @@ type serverSuite struct { errg *errgroup.Group } -func (s *serverSuite) SetUpTest(c *check.C) { +func newServer(t *testing.T) *testServer { var err error - dir := c.MkDir() + dir := t.TempDir() + s := &testServer{} s.clientURL, s.e, err = etcd.SetupEmbedEtcd(dir) - c.Assert(err, check.IsNil) + require.Nil(t, err) pdEndpoints := []string{ "http://" + s.clientURL.Host, "http://invalid-pd-host:2379", } server, err := NewServer(pdEndpoints) - c.Assert(err, check.IsNil) - c.Assert(server, check.NotNil) + require.Nil(t, err) + require.NotNil(t, server) s.server = server s.ctx, s.cancel = context.WithCancel(context.Background()) @@ -65,78 +75,75 @@ func (s *serverSuite) SetUpTest(c *check.C) { Context: s.ctx, DialTimeout: 5 * time.Second, }) - c.Assert(err, check.IsNil) + require.Nil(t, err) etcdClient := etcd.NewCDCEtcdClient(s.ctx, client) s.server.etcdClient = &etcdClient - s.errg = util.HandleErrWithErrGroup(s.ctx, s.e.Err(), func(e error) { c.Log(e) }) + s.errg = util.HandleErrWithErrGroup(s.ctx, s.e.Err(), func(e error) { t.Log(e) }) + return s } -func (s *serverSuite) TearDownTest(c *check.C) { +func (s *testServer) close(t *testing.T) { s.server.Close() s.e.Close() s.cancel() err := s.errg.Wait() if err != nil { - c.Errorf("Error group error: %s", err) + t.Errorf("Error group error: %s", err) } } -var _ = check.Suite(&serverSuite{}) - -func (s *serverSuite) TestEtcdHealthChecker(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) +func TestServerBasic(t *testing.T) { + t.Parallel() + s := newServer(t) + defer s.close(t) + testEtcdHealthChecker(t, s) + testSetUpDataDir(t, s) +} +func testEtcdHealthChecker(t *testing.T, s *testServer) { s.errg.Go(func() error { err := s.server.etcdHealthChecker(s.ctx) - c.Assert(err, check.Equals, context.Canceled) + require.Equal(t, context.Canceled, err) return nil }) // longer than one check tick 3s time.Sleep(time.Second * 4) - s.cancel() } -func (s *serverSuite) TestSetUpDataDir(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - +func testSetUpDataDir(t *testing.T, s *testServer) { conf := config.GetGlobalServerConfig() // DataDir is not set, and no changefeed exist, use the default conf.DataDir = "" err := s.server.setUpDir(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(conf.DataDir, check.Equals, defaultDataDir) - c.Assert(conf.Sorter.SortDir, check.Equals, filepath.Join(defaultDataDir, config.DefaultSortDir)) + require.Nil(t, err) + require.Equal(t, defaultDataDir, conf.DataDir) + require.Equal(t, filepath.Join(defaultDataDir, config.DefaultSortDir), conf.Sorter.SortDir) // DataDir is not set, but has existed changefeed, use the one with the largest available space conf.DataDir = "" - dir := c.MkDir() + dir := t.TempDir() err = s.server.etcdClient.SaveChangeFeedInfo(s.ctx, &model.ChangeFeedInfo{SortDir: dir}, "a") - c.Assert(err, check.IsNil) + require.Nil(t, err) err = s.server.etcdClient.SaveChangeFeedInfo(s.ctx, &model.ChangeFeedInfo{}, "b") - c.Assert(err, check.IsNil) + require.Nil(t, err) err = s.server.setUpDir(s.ctx) - c.Assert(err, check.IsNil) + require.Nil(t, err) - c.Assert(conf.DataDir, check.Equals, dir) - c.Assert(conf.Sorter.SortDir, check.Equals, filepath.Join(dir, config.DefaultSortDir)) + require.Equal(t, dir, conf.DataDir) + require.Equal(t, filepath.Join(dir, config.DefaultSortDir), conf.Sorter.SortDir) - conf.DataDir = c.MkDir() + conf.DataDir = t.TempDir() // DataDir has been set, just use it err = s.server.setUpDir(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(conf.DataDir, check.Not(check.Equals), "") - c.Assert(conf.Sorter.SortDir, check.Equals, filepath.Join(conf.DataDir, config.DefaultSortDir)) - - s.cancel() + require.Nil(t, err) + require.NotEqual(t, "", conf.DataDir) + require.Equal(t, filepath.Join(conf.DataDir, config.DefaultSortDir), conf.Sorter.SortDir) } func TestCheckDir(t *testing.T) { - t.Parallel() me, err := user.Current() require.Nil(t, err) if me.Name == "root" || runtime.GOOS == "windows" { @@ -166,3 +173,154 @@ func TestCheckDir(t *testing.T) { _, err = checkDir(file) require.Error(t, err) } + +const retryTime = 20 + +func TestServerTLSWithoutCommonName(t *testing.T) { + addr := tempurl.Alloc()[len("http://"):] + // Do not specify common name + security, err := security2.NewCredential4Test("") + require.Nil(t, err) + conf := config.GetDefaultServerConfig() + conf.Addr = addr + conf.AdvertiseAddr = addr + conf.Security = &security + config.StoreGlobalServerConfig(conf) + + server, err := NewServer([]string{"https://127.0.0.1:2379"}) + server.capture = capture.NewCapture4Test(false) + require.Nil(t, err) + err = server.startStatusHTTP(server.tcpServer.HTTP1Listener()) + require.Nil(t, err) + defer func() { + require.Nil(t, server.statusServer.Close()) + }() + + statusURL := fmt.Sprintf("https://%s/api/v1/status", addr) + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + err := server.tcpServer.Run(ctx) + require.Contains(t, err.Error(), "ErrTCPServerClosed") + }() + + // test cli sends request without a cert will success + err = retry.Do(ctx, func() error { + tr := &http.Transport{ + TLSClientConfig: &tls.Config{InsecureSkipVerify: true}, + } + cli := &http.Client{Transport: tr} + resp, err := cli.Get(statusURL) + if err != nil { + return err + } + decoder := json.NewDecoder(resp.Body) + captureInfo := &model.CaptureInfo{} + err = decoder.Decode(captureInfo) + require.Nil(t, err) + require.Equal(t, server.capture.Info().ID, captureInfo.ID) + resp.Body.Close() + return nil + }, retry.WithMaxTries(retryTime), retry.WithBackoffBaseDelay(50), retry.WithIsRetryableErr(cerrors.IsRetryableError)) + require.Nil(t, err) + + // test cli sends request with a cert will success + err = retry.Do(ctx, func() error { + tlsConfig, err := security.ToTLSConfigWithVerify() + require.Nil(t, err) + + cli := httputil.NewClient(tlsConfig) + resp, err := cli.Get(statusURL) + if err != nil { + return err + } + decoder := json.NewDecoder(resp.Body) + captureInfo := &model.CaptureInfo{} + err = decoder.Decode(captureInfo) + require.Nil(t, err) + require.Equal(t, server.capture.Info().ID, captureInfo.ID) + resp.Body.Close() + return nil + }, retry.WithMaxTries(retryTime), retry.WithBackoffBaseDelay(50), retry.WithIsRetryableErr(cerrors.IsRetryableError)) + require.Nil(t, err) + + cancel() + wg.Wait() +} + +func TestServerTLSWithCommonName(t *testing.T) { + addr := tempurl.Alloc()[len("http://"):] + // specify a common name + security, err := security2.NewCredential4Test("test") + require.Nil(t, err) + conf := config.GetDefaultServerConfig() + conf.Addr = addr + conf.AdvertiseAddr = addr + conf.Security = &security + config.StoreGlobalServerConfig(conf) + + server, err := NewServer([]string{"https://127.0.0.1:2379"}) + server.capture = capture.NewCapture4Test(false) + require.Nil(t, err) + err = server.startStatusHTTP(server.tcpServer.HTTP1Listener()) + require.Nil(t, err) + defer func() { + require.Nil(t, server.statusServer.Close()) + }() + + statusURL := fmt.Sprintf("https://%s/api/v1/status", addr) + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + err := server.tcpServer.Run(ctx) + require.Contains(t, err.Error(), "ErrTCPServerClosed") + }() + + // test cli sends request without a cert will fail + err = retry.Do(ctx, func() error { + tr := &http.Transport{ + TLSClientConfig: &tls.Config{InsecureSkipVerify: true}, + } + cli := &http.Client{Transport: tr} + resp, err := cli.Get(statusURL) + if err != nil { + return err + } + decoder := json.NewDecoder(resp.Body) + captureInfo := &model.CaptureInfo{} + err = decoder.Decode(captureInfo) + require.Nil(t, err) + require.Equal(t, server.capture.Info().ID, captureInfo.ID) + resp.Body.Close() + return nil + }, retry.WithMaxTries(retryTime), retry.WithBackoffBaseDelay(50), retry.WithIsRetryableErr(cerrors.IsRetryableError)) + require.Contains(t, err.Error(), "remote error: tls: bad certificate") + + // test cli sends request with a cert will success + err = retry.Do(ctx, func() error { + tlsConfig, err := security.ToTLSConfigWithVerify() + require.Nil(t, err) + + cli := httputil.NewClient(tlsConfig) + resp, err := cli.Get(statusURL) + if err != nil { + return err + } + decoder := json.NewDecoder(resp.Body) + captureInfo := &model.CaptureInfo{} + err = decoder.Decode(captureInfo) + require.Nil(t, err) + require.Equal(t, server.capture.Info().ID, captureInfo.ID) + resp.Body.Close() + return nil + }, retry.WithMaxTries(retryTime), retry.WithBackoffBaseDelay(50), retry.WithIsRetryableErr(cerrors.IsRetryableError)) + require.Nil(t, err) +} diff --git a/cdc/sink/dispatcher/default_test.go b/cdc/sink/dispatcher/default_test.go index 93fe3952af1..ccddd3052e6 100644 --- a/cdc/sink/dispatcher/default_test.go +++ b/cdc/sink/dispatcher/default_test.go @@ -14,17 +14,15 @@ package dispatcher import ( - "github.com/pingcap/check" + "testing" + "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) -type DefaultDispatcherSuite struct{} - -var _ = check.Suite(&DefaultDispatcherSuite{}) +func TestDefaultDispatcher(t *testing.T) { + t.Parallel() -func (s DefaultDispatcherSuite) TestDefaultDispatcher(c *check.C) { - defer testleak.AfterTest(c)() testCases := []struct { row *model.RowChangedEvent exceptPartition int32 @@ -197,6 +195,32 @@ func (s DefaultDispatcherSuite) TestDefaultDispatcher(c *check.C) { } p := newDefaultDispatcher(16, false) for _, tc := range testCases { - c.Assert(p.Dispatch(tc.row), check.Equals, tc.exceptPartition) + require.Equal(t, tc.exceptPartition, p.Dispatch(tc.row)) } } + +func TestDefaultDispatcherWithOldValue(t *testing.T) { + t.Parallel() + + row := &model.RowChangedEvent{ + Table: &model.TableName{ + Schema: "test", + Table: "t3", + }, + Columns: []*model.Column{ + { + Name: "id", + Value: 2, + Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, + }, { + Name: "a", + Value: 3, + Flag: model.UniqueKeyFlag, + }, + }, + IndexColumns: [][]int{{0}, {1}}, + } + + p := newDefaultDispatcher(16, true) + require.Equal(t, int32(3), p.Dispatch(row)) +} diff --git a/cdc/sink/dispatcher/index_value_test.go b/cdc/sink/dispatcher/index_value_test.go index 5f0890885c2..b52ed548883 100644 --- a/cdc/sink/dispatcher/index_value_test.go +++ b/cdc/sink/dispatcher/index_value_test.go @@ -14,17 +14,15 @@ package dispatcher import ( - "github.com/pingcap/check" + "testing" + "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) -type IndexValueDispatcherSuite struct{} - -var _ = check.Suite(&IndexValueDispatcherSuite{}) +func TestIndexValueDispatcher(t *testing.T) { + t.Parallel() -func (s IndexValueDispatcherSuite) TestIndexValueDispatcher(c *check.C) { - defer testleak.AfterTest(c)() testCases := []struct { row *model.RowChangedEvent exceptPartition int32 @@ -151,6 +149,6 @@ func (s IndexValueDispatcherSuite) TestIndexValueDispatcher(c *check.C) { } p := newIndexValueDispatcher(16) for _, tc := range testCases { - c.Assert(p.Dispatch(tc.row), check.Equals, tc.exceptPartition) + require.Equal(t, tc.exceptPartition, p.Dispatch(tc.row)) } } diff --git a/cdc/sink/dispatcher/switcher_test.go b/cdc/sink/dispatcher/switcher_test.go index 6f3caf20143..ca3eeab003a 100644 --- a/cdc/sink/dispatcher/switcher_test.go +++ b/cdc/sink/dispatcher/switcher_test.go @@ -14,30 +14,29 @@ package dispatcher import ( - "github.com/pingcap/check" + "testing" + "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) -type SwitcherSuite struct{} - -var _ = check.Suite(&SwitcherSuite{}) +func TestSwitcher(t *testing.T) { + t.Parallel() -func (s SwitcherSuite) TestSwitcher(c *check.C) { - defer testleak.AfterTest(c)() d, err := NewDispatcher(config.GetDefaultReplicaConfig(), 4) - c.Assert(err, check.IsNil) - c.Assert(d.(*dispatcherSwitcher).matchDispatcher(&model.RowChangedEvent{ + require.Nil(t, err) + require.IsType(t, &defaultDispatcher{}, d.(*dispatcherSwitcher).matchDispatcher(&model.RowChangedEvent{ Table: &model.TableName{ Schema: "test", Table: "test", }, - }), check.FitsTypeOf, &defaultDispatcher{}) + })) d, err = NewDispatcher(&config.ReplicaConfig{ Sink: &config.SinkConfig{ DispatchRules: []*config.DispatchRule{ - {Matcher: []string{"test_default.*"}, Dispatcher: "default"}, + {Matcher: []string{"test_default1.*"}, Dispatcher: "default"}, + {Matcher: []string{"test_default2.*"}, Dispatcher: "unknown-dispatcher"}, {Matcher: []string{"test_table.*"}, Dispatcher: "table"}, {Matcher: []string{"test_index_value.*"}, Dispatcher: "index-value"}, {Matcher: []string{"test.*"}, Dispatcher: "rowid"}, @@ -45,35 +44,40 @@ func (s SwitcherSuite) TestSwitcher(c *check.C) { }, }, }, 4) - c.Assert(err, check.IsNil) - c.Assert(d.(*dispatcherSwitcher).matchDispatcher(&model.RowChangedEvent{ + require.Nil(t, err) + require.IsType(t, &indexValueDispatcher{}, d.(*dispatcherSwitcher).matchDispatcher(&model.RowChangedEvent{ Table: &model.TableName{ Schema: "test", Table: "table1", }, - }), check.FitsTypeOf, &indexValueDispatcher{}) - c.Assert(d.(*dispatcherSwitcher).matchDispatcher(&model.RowChangedEvent{ + })) + require.IsType(t, &tsDispatcher{}, d.(*dispatcherSwitcher).matchDispatcher(&model.RowChangedEvent{ Table: &model.TableName{ Schema: "sbs", Table: "table2", }, - }), check.FitsTypeOf, &tsDispatcher{}) - c.Assert(d.(*dispatcherSwitcher).matchDispatcher(&model.RowChangedEvent{ + })) + require.IsType(t, &defaultDispatcher{}, d.(*dispatcherSwitcher).matchDispatcher(&model.RowChangedEvent{ Table: &model.TableName{ Schema: "sbs", Table: "test", }, - }), check.FitsTypeOf, &defaultDispatcher{}) - c.Assert(d.(*dispatcherSwitcher).matchDispatcher(&model.RowChangedEvent{ + })) + require.IsType(t, &defaultDispatcher{}, d.(*dispatcherSwitcher).matchDispatcher(&model.RowChangedEvent{ + Table: &model.TableName{ + Schema: "test_default1", Table: "test", + }, + })) + require.IsType(t, &defaultDispatcher{}, d.(*dispatcherSwitcher).matchDispatcher(&model.RowChangedEvent{ Table: &model.TableName{ - Schema: "test_default", Table: "test", + Schema: "test_default2", Table: "test", }, - }), check.FitsTypeOf, &defaultDispatcher{}) - c.Assert(d.(*dispatcherSwitcher).matchDispatcher(&model.RowChangedEvent{ + })) + require.IsType(t, &tableDispatcher{}, d.(*dispatcherSwitcher).matchDispatcher(&model.RowChangedEvent{ Table: &model.TableName{ Schema: "test_table", Table: "test", }, - }), check.FitsTypeOf, &tableDispatcher{}) - c.Assert(d.(*dispatcherSwitcher).matchDispatcher(&model.RowChangedEvent{ + })) + require.IsType(t, &indexValueDispatcher{}, d.(*dispatcherSwitcher).matchDispatcher(&model.RowChangedEvent{ Table: &model.TableName{ Schema: "test_index_value", Table: "test", }, - }), check.FitsTypeOf, &indexValueDispatcher{}) + })) } diff --git a/cdc/sink/dispatcher/table_test.go b/cdc/sink/dispatcher/table_test.go index 9b2f17c4108..d0bb114968b 100644 --- a/cdc/sink/dispatcher/table_test.go +++ b/cdc/sink/dispatcher/table_test.go @@ -14,17 +14,15 @@ package dispatcher import ( - "github.com/pingcap/check" + "testing" + "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) -type TableDispatcherSuite struct{} - -var _ = check.Suite(&TableDispatcherSuite{}) +func TestTableDispatcher(t *testing.T) { + t.Parallel() -func (s TableDispatcherSuite) TestTableDispatcher(c *check.C) { - defer testleak.AfterTest(c)() testCases := []struct { row *model.RowChangedEvent exceptPartition int32 @@ -81,6 +79,6 @@ func (s TableDispatcherSuite) TestTableDispatcher(c *check.C) { } p := newTableDispatcher(16) for _, tc := range testCases { - c.Assert(p.Dispatch(tc.row), check.Equals, tc.exceptPartition) + require.Equal(t, tc.exceptPartition, p.Dispatch(tc.row)) } } diff --git a/cdc/sink/dispatcher/ts_test.go b/cdc/sink/dispatcher/ts_test.go index f4eac2471bb..a5f33019f9f 100644 --- a/cdc/sink/dispatcher/ts_test.go +++ b/cdc/sink/dispatcher/ts_test.go @@ -16,19 +16,13 @@ package dispatcher import ( "testing" - "github.com/pingcap/check" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) -func Test(t *testing.T) { check.TestingT(t) } +func TestTsDispatcher(t *testing.T) { + t.Parallel() -type TsDispatcherSuite struct{} - -var _ = check.Suite(&TsDispatcherSuite{}) - -func (s TsDispatcherSuite) TestTsDispatcher(c *check.C) { - defer testleak.AfterTest(c)() testCases := []struct { row *model.RowChangedEvent exceptPartition int32 @@ -78,6 +72,6 @@ func (s TsDispatcherSuite) TestTsDispatcher(c *check.C) { } p := &tsDispatcher{partitionNum: 16} for _, tc := range testCases { - c.Assert(p.Dispatch(tc.row), check.Equals, tc.exceptPartition) + require.Equal(t, tc.exceptPartition, p.Dispatch(tc.row)) } } diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index afca2bc9a7d..898cc45a204 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -17,6 +17,7 @@ import ( "context" "net/url" "strings" + "sync" "sync/atomic" "time" @@ -39,11 +40,21 @@ import ( type mqEvent struct { row *model.RowChangedEvent - resolvedTs uint64 + resolvedTs model.Ts +} + +type resolvedTsEvent struct { + tableID model.TableID + resolvedTs model.Ts } const ( - defaultPartitionInputChSize = 12800 + // Depend on this size, every `partitionInputCh` will take + // approximately 16.3 KiB memory. + defaultPartitionInputChSize = 1024 + // Depend on this size, `resolvedBuffer` will take + // approximately 2 KiB memory. + defaultResolvedTsEventBufferSize = 128 // -1 means broadcast to all partitions, it's the default for the default open protocol. defaultDDLDispatchPartition = -1 ) @@ -55,12 +66,13 @@ type mqSink struct { filter *filter.Filter protocol config.Protocol - partitionNum int32 - partitionInput []chan mqEvent - partitionResolvedTs []uint64 - tableCheckpointTs map[model.TableID]uint64 - resolvedNotifier *notify.Notifier - resolvedReceiver *notify.Receiver + partitionNum int32 + partitionInput []chan mqEvent + partitionResolvedTs []uint64 + tableCheckpointTsMap sync.Map + resolvedBuffer chan resolvedTsEvent + resolvedNotifier *notify.Notifier + resolvedReceiver *notify.Receiver statistics *Statistics } @@ -110,9 +122,10 @@ func newMqSink( partitionNum: partitionNum, partitionInput: partitionInput, partitionResolvedTs: make([]uint64, partitionNum), - tableCheckpointTs: make(map[model.TableID]uint64), - resolvedNotifier: notifier, - resolvedReceiver: resolvedReceiver, + + resolvedBuffer: make(chan resolvedTsEvent, defaultResolvedTsEventBufferSize), + resolvedNotifier: notifier, + resolvedReceiver: resolvedReceiver, statistics: NewStatistics(ctx, "MQ", opts), } @@ -164,19 +177,59 @@ func (k *mqSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowCha return nil } +// FlushRowChangedEvents is thread-safety func (k *mqSink) FlushRowChangedEvents(ctx context.Context, tableID model.TableID, resolvedTs uint64) (uint64, error) { - if checkpointTs, ok := k.tableCheckpointTs[tableID]; ok && resolvedTs <= checkpointTs { + var checkpointTs uint64 + v, ok := k.tableCheckpointTsMap.Load(tableID) + if ok { + checkpointTs = v.(uint64) + } + if resolvedTs <= checkpointTs { return checkpointTs, nil } + select { + case <-ctx.Done(): + return 0, ctx.Err() + case k.resolvedBuffer <- resolvedTsEvent{ + tableID: tableID, + resolvedTs: resolvedTs, + }: + } + k.statistics.PrintStatus(ctx) + return checkpointTs, nil +} + +// bgFlushTs flush resolvedTs to workers and flush the mqProducer +func (k *mqSink) bgFlushTs(ctx context.Context) error { + for { + select { + case <-ctx.Done(): + return errors.Trace(ctx.Err()) + case msg := <-k.resolvedBuffer: + resolvedTs := msg.resolvedTs + err := k.flushTsToWorker(ctx, resolvedTs) + if err != nil { + return errors.Trace(err) + } + err = k.mqProducer.Flush(ctx) + if err != nil { + return errors.Trace(err) + } + // Since CDC does not guarantee exactly once semantic, it won't cause any problem + // here even if the table was moved or removed. + // ref: https://github.com/pingcap/tiflow/pull/4356#discussion_r787405134 + k.tableCheckpointTsMap.Store(msg.tableID, resolvedTs) + } + } +} +func (k *mqSink) flushTsToWorker(ctx context.Context, resolvedTs model.Ts) error { + // flush resolvedTs to all partition workers for i := 0; i < int(k.partitionNum); i++ { select { case <-ctx.Done(): - return 0, ctx.Err() - case k.partitionInput[i] <- struct { - row *model.RowChangedEvent - resolvedTs uint64 - }{resolvedTs: resolvedTs}: + return errors.Trace(ctx.Err()) + case k.partitionInput[i] <- mqEvent{resolvedTs: resolvedTs}: } } @@ -185,23 +238,16 @@ flushLoop: for { select { case <-ctx.Done(): - return 0, ctx.Err() + return errors.Trace(ctx.Err()) case <-k.resolvedReceiver.C: for i := 0; i < int(k.partitionNum); i++ { if resolvedTs > atomic.LoadUint64(&k.partitionResolvedTs[i]) { continue flushLoop } } - break flushLoop + return nil } } - err := k.mqProducer.Flush(ctx) - if err != nil { - return 0, errors.Trace(err) - } - k.tableCheckpointTs[tableID] = resolvedTs - k.statistics.PrintStatus(ctx) - return resolvedTs, nil } func (k *mqSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { @@ -273,6 +319,9 @@ func (k *mqSink) Barrier(cxt context.Context, tableID model.TableID) error { func (k *mqSink) run(ctx context.Context) error { defer k.resolvedReceiver.Stop() wg, ctx := errgroup.WithContext(ctx) + wg.Go(func() error { + return k.bgFlushTs(ctx) + }) for i := int32(0); i < k.partitionNum; i++ { partition := i wg.Go(func() error { @@ -331,6 +380,7 @@ func (k *mqSink) runWorker(ctx context.Context, partition int32) error { continue case e = <-input: } + // flush resolvedTs event if e.row == nil { if e.resolvedTs != 0 { op, err := encoder.AppendResolvedEvent(e.resolvedTs) diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index 081ad8bd2c6..7b81d0a24b0 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -116,6 +116,7 @@ func newMySQLSink( } // create test db used for parameter detection + // Refer https://github.com/go-sql-driver/mysql#parameters if dsn.Params == nil { dsn.Params = make(map[string]string, 1) } @@ -224,6 +225,9 @@ func (s *mysqlSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.Row // FlushRowChangedEvents will flush all received events, we don't allow mysql // sink to receive events before resolving func (s *mysqlSink) FlushRowChangedEvents(ctx context.Context, tableID model.TableID, resolvedTs uint64) (uint64, error) { + // Since CDC does not guarantee exactly once semantic, it won't cause any problem + // here even if the table was moved or removed. + // ref: https://github.com/pingcap/tiflow/pull/4356#discussion_r787405134 v, ok := s.tableMaxResolvedTs.Load(tableID) if !ok || v.(uint64) < resolvedTs { s.tableMaxResolvedTs.Store(tableID, resolvedTs) @@ -674,8 +678,6 @@ func (s *mysqlSink) prepareDMLs(rows []*model.RowChangedEvent, replicaID uint64, } } else { query, args = prepareReplace(quoteTable, row.Columns, true /* appendPlaceHolder */, translateToInsert) - sqls = append(sqls, query) - values = append(values, args) if query != "" { sqls = append(sqls, query) values = append(values, args) diff --git a/cdc/sink/mysql_params.go b/cdc/sink/mysql_params.go index a374dbb2538..9f0fec92a41 100644 --- a/cdc/sink/mysql_params.go +++ b/cdc/sink/mysql_params.go @@ -35,6 +35,10 @@ const ( // expose these two variables for redo log applier DefaultWorkerCount = 16 DefaultMaxTxnRow = 256 + // The upper limit of max worker counts. + maxWorkerCount = 1024 + // The upper limit of max txn rows. + maxMaxTxnRow = 2048 defaultDMLMaxRetryTime = 8 defaultDDLMaxRetryTime = 20 @@ -113,9 +117,16 @@ func parseSinkURIToParams(ctx context.Context, sinkURI *url.URL, opts map[string if err != nil { return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) } - if c > 0 { - params.workerCount = c + if c <= 0 { + return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, + fmt.Errorf("invalid worker-count %d, which must be greater than 0", c)) } + if c > maxWorkerCount { + log.Warn("worker-count too large", + zap.Int("original", c), zap.Int("override", maxWorkerCount)) + c = maxWorkerCount + } + params.workerCount = c } s = sinkURI.Query().Get("max-txn-row") if s != "" { @@ -123,6 +134,15 @@ func parseSinkURIToParams(ctx context.Context, sinkURI *url.URL, opts map[string if err != nil { return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) } + if c <= 0 { + return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, + fmt.Errorf("invalid max-txn-row %d, which must be greater than 0", c)) + } + if c > maxMaxTxnRow { + log.Warn("max-txn-row too large", + zap.Int("original", c), zap.Int("override", maxMaxTxnRow)) + c = maxMaxTxnRow + } params.maxTxnRow = c } s = sinkURI.Query().Get("tidb-txn-mode") @@ -182,6 +202,14 @@ func parseSinkURIToParams(ctx context.Context, sinkURI *url.URL, opts map[string if s == "" { params.timezone = "" } else { + value, err := url.QueryUnescape(s) + if err != nil { + return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) + } + _, err = time.LoadLocation(value) + if err != nil { + return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) + } params.timezone = fmt.Sprintf(`"%s"`, s) } } else { @@ -195,14 +223,26 @@ func parseSinkURIToParams(ctx context.Context, sinkURI *url.URL, opts map[string // To keep the same style with other sink parameters, we use dash as word separator. s = sinkURI.Query().Get("read-timeout") if s != "" { + _, err := time.ParseDuration(s) + if err != nil { + return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) + } params.readTimeout = s } s = sinkURI.Query().Get("write-timeout") if s != "" { + _, err := time.ParseDuration(s) + if err != nil { + return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) + } params.writeTimeout = s } s = sinkURI.Query().Get("timeout") if s != "" { + _, err := time.ParseDuration(s) + if err != nil { + return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) + } params.dialTimeout = s } diff --git a/cdc/sink/mysql_params_test.go b/cdc/sink/mysql_params_test.go index 91f8a0e6d00..075e5d41d26 100644 --- a/cdc/sink/mysql_params_test.go +++ b/cdc/sink/mysql_params_test.go @@ -172,17 +172,63 @@ func TestParseSinkURITimezone(t *testing.T) { } } +func TestParseSinkURIOverride(t *testing.T) { + defer testleak.AfterTestT(t)() + cases := []struct { + uri string + checker func(*sinkParams) + }{{ + uri: "mysql://127.0.0.1:3306/?worker-count=2147483648", // int32 max + checker: func(sp *sinkParams) { + require.EqualValues(t, sp.workerCount, maxWorkerCount) + }, + }, { + uri: "mysql://127.0.0.1:3306/?max-txn-row=2147483648", // int32 max + checker: func(sp *sinkParams) { + require.EqualValues(t, sp.maxTxnRow, maxMaxTxnRow) + }, + }, { + uri: "mysql://127.0.0.1:3306/?tidb-txn-mode=badmode", + checker: func(sp *sinkParams) { + require.EqualValues(t, sp.tidbTxnMode, defaultTiDBTxnMode) + }, + }} + ctx := context.TODO() + opts := map[string]string{OptChangefeedID: "changefeed-01"} + var uri *url.URL + var err error + for _, cs := range cases { + if cs.uri != "" { + uri, err = url.Parse(cs.uri) + require.Nil(t, err) + } else { + uri = nil + } + p, err := parseSinkURIToParams(ctx, uri, opts) + require.Nil(t, err) + cs.checker(p) + } +} + func TestParseSinkURIBadQueryString(t *testing.T) { defer testleak.AfterTestT(t)() uris := []string{ "", "postgre://127.0.0.1:3306", "mysql://127.0.0.1:3306/?worker-count=not-number", + "mysql://127.0.0.1:3306/?worker-count=-1", + "mysql://127.0.0.1:3306/?worker-count=0", "mysql://127.0.0.1:3306/?max-txn-row=not-number", + "mysql://127.0.0.1:3306/?max-txn-row=-1", + "mysql://127.0.0.1:3306/?max-txn-row=0", "mysql://127.0.0.1:3306/?ssl-ca=only-ca-exists", "mysql://127.0.0.1:3306/?batch-replace-enable=not-bool", "mysql://127.0.0.1:3306/?batch-replace-enable=true&batch-replace-size=not-number", "mysql://127.0.0.1:3306/?safe-mode=not-bool", + "mysql://127.0.0.1:3306/?time-zone=badtz", + "mysql://127.0.0.1:3306/?write-timeout=badduration", + "mysql://127.0.0.1:3306/?read-timeout=badduration", + "mysql://127.0.0.1:3306/?timeout=badduration", } ctx := context.TODO() opts := map[string]string{OptChangefeedID: "changefeed-01"} @@ -196,7 +242,7 @@ func TestParseSinkURIBadQueryString(t *testing.T) { uri = nil } _, err = parseSinkURIToParams(ctx, uri, opts) - require.NotNil(t, err) + require.Error(t, err) } } diff --git a/cdc/sink/mysql_test.go b/cdc/sink/mysql_test.go index cf42a6cb8c8..cc0b119350c 100644 --- a/cdc/sink/mysql_test.go +++ b/cdc/sink/mysql_test.go @@ -58,41 +58,68 @@ func TestPrepareDML(t *testing.T) { testCases := []struct { input []*model.RowChangedEvent expected *preparedDMLs - }{{ - input: []*model.RowChangedEvent{}, - expected: &preparedDMLs{sqls: []string{}, values: [][]interface{}{}}, - }, { - input: []*model.RowChangedEvent{ - { - StartTs: 418658114257813514, - CommitTs: 418658114257813515, - Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, - PreColumns: []*model.Column{nil, { - Name: "a1", - Type: mysql.TypeLong, - Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, - Value: 1, - }, { - Name: "a3", - Type: mysql.TypeLong, - Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, - Value: 1, - }}, - IndexColumns: [][]int{{1, 2}}, + }{ + { + input: []*model.RowChangedEvent{}, + expected: &preparedDMLs{sqls: []string{}, values: [][]interface{}{}}, + }, { + input: []*model.RowChangedEvent{ + { + StartTs: 418658114257813514, + CommitTs: 418658114257813515, + Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + PreColumns: []*model.Column{nil, { + Name: "a1", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, + Value: 1, + }, { + Name: "a3", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, + Value: 1, + }}, + IndexColumns: [][]int{{1, 2}}, + }, + }, + expected: &preparedDMLs{ + sqls: []string{"DELETE FROM `common_1`.`uk_without_pk` WHERE `a1` = ? AND `a3` = ? LIMIT 1;"}, + values: [][]interface{}{{1, 1}}, + rowCount: 1, + }, + }, { + input: []*model.RowChangedEvent{ + { + StartTs: 418658114257813516, + CommitTs: 418658114257813517, + Table: &model.TableName{Schema: "common_1", Table: "uk_without_pk"}, + Columns: []*model.Column{nil, { + Name: "a1", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, + Value: 2, + }, { + Name: "a3", + Type: mysql.TypeLong, + Flag: model.BinaryFlag | model.MultipleKeyFlag | model.HandleKeyFlag, + Value: 2, + }}, + IndexColumns: [][]int{{1, 2}}, + }, + }, + expected: &preparedDMLs{ + sqls: []string{"REPLACE INTO `common_1`.`uk_without_pk`(`a1`,`a3`) VALUES (?,?);"}, + values: [][]interface{}{{2, 2}}, + rowCount: 1, }, }, - expected: &preparedDMLs{ - sqls: []string{"DELETE FROM `common_1`.`uk_without_pk` WHERE `a1` = ? AND `a3` = ? LIMIT 1;"}, - values: [][]interface{}{{1, 1}}, - rowCount: 1, - }, - }} + } ctx, cancel := context.WithCancel(context.Background()) defer cancel() ms := newMySQLSink4Test(ctx, t) - for i, tc := range testCases { + for _, tc := range testCases { dmls := ms.prepareDMLs(tc.input, 0, 0) - require.Equal(t, tc.expected, dmls, tc.expected, fmt.Sprintf("%d", i)) + require.Equal(t, tc.expected, dmls) } } diff --git a/cdc/sorter/unified/merger.go b/cdc/sorter/unified/merger.go index b3fe8cab196..74bfa9193d5 100644 --- a/cdc/sorter/unified/merger.go +++ b/cdc/sorter/unified/merger.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sorter" cerrors "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/notify" "github.com/pingcap/tiflow/pkg/util" "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" @@ -405,8 +404,7 @@ func runMerger(ctx context.Context, numSorters int, in <-chan *flushTask, out ch return nil } - resolvedTsNotifier := ¬ify.Notifier{} - defer resolvedTsNotifier.Close() + resolvedTsNotifierChan := make(chan struct{}, 1) errg, ctx := errgroup.WithContext(ctx) errg.Go(func() error { @@ -443,40 +441,46 @@ func runMerger(ctx context.Context, numSorters int, in <-chan *flushTask, out ch if minTemp > minResolvedTs { atomic.StoreUint64(&minResolvedTs, minTemp) - resolvedTsNotifier.Notify() + select { + case resolvedTsNotifierChan <- struct{}{}: + default: + } } } }) errg.Go(func() error { - resolvedTsReceiver, err := resolvedTsNotifier.NewReceiver(time.Second * 1) - if err != nil { - if cerrors.ErrOperateOnClosedNotifier.Equal(err) { - // This won't happen unless `resolvedTsNotifier` has been closed, which is - // impossible at this point. - log.Panic("unexpected error", zap.Error(err)) - } - return errors.Trace(err) - } + resolvedTsTicker := time.NewTicker(time.Second * 1) - defer resolvedTsReceiver.Stop() + defer resolvedTsTicker.Stop() var lastResolvedTs uint64 + resolvedTsTickFunc := func() error { + curResolvedTs := atomic.LoadUint64(&minResolvedTs) + if curResolvedTs > lastResolvedTs { + err := onMinResolvedTsUpdate(curResolvedTs) + if err != nil { + return errors.Trace(err) + } + } else if curResolvedTs < lastResolvedTs { + log.Panic("resolved-ts regressed in sorter", + zap.Uint64("curResolvedTs", curResolvedTs), + zap.Uint64("lastResolvedTs", lastResolvedTs)) + } + return nil + } + for { select { case <-ctx.Done(): return ctx.Err() - case <-resolvedTsReceiver.C: - curResolvedTs := atomic.LoadUint64(&minResolvedTs) - if curResolvedTs > lastResolvedTs { - err := onMinResolvedTsUpdate(curResolvedTs) - if err != nil { - return errors.Trace(err) - } - } else if curResolvedTs < lastResolvedTs { - log.Panic("resolved-ts regressed in sorter", - zap.Uint64("curResolved-ts", curResolvedTs), - zap.Uint64("lastResolved-ts", lastResolvedTs)) + case <-resolvedTsTicker.C: + if err := resolvedTsTickFunc(); err != nil { + return err + } + case <-resolvedTsNotifierChan: + if err := resolvedTsTickFunc(); err != nil { + return err } } } diff --git a/cdc/sorter/unified/unified_sorter.go b/cdc/sorter/unified/unified_sorter.go index e8c1770a8f9..42a0394e0ec 100644 --- a/cdc/sorter/unified/unified_sorter.go +++ b/cdc/sorter/unified/unified_sorter.go @@ -114,7 +114,7 @@ func (s *Sorter) Run(ctx context.Context) error { defer close(s.closeCh) - finish := util.MonitorCancelLatency(ctx, "Unified Sorter") + finish, startCancel := util.MonitorCancelLatency(ctx, "Unified Sorter") defer finish() ctx = context.WithValue(ctx, ctxKey{}, s) @@ -163,6 +163,7 @@ func (s *Sorter) Run(ctx context.Context) error { select { case <-subctx.Done(): + startCancel() return errors.Trace(subctx.Err()) case err := <-heapSorterErrCh: return errors.Trace(err) diff --git a/cmd/kafka-consumer/main.go b/cmd/kafka-consumer/main.go index ab71f256bb9..659e106323f 100644 --- a/cmd/kafka-consumer/main.go +++ b/cmd/kafka-consumer/main.go @@ -80,16 +80,16 @@ func init() { File: logPath, }) if err != nil { - log.Fatal("init logger failed", zap.Error(err)) + log.Panic("init logger failed", zap.Error(err)) } upstreamURI, err := url.Parse(upstreamURIStr) if err != nil { - log.Fatal("invalid upstream-uri", zap.Error(err)) + log.Panic("invalid upstream-uri", zap.Error(err)) } scheme := strings.ToLower(upstreamURI.Scheme) if scheme != "kafka" { - log.Fatal("invalid upstream-uri scheme, the scheme of upstream-uri must be `kafka`", + log.Panic("invalid upstream-uri scheme, the scheme of upstream-uri must be `kafka`", zap.String("upstreamURI", upstreamURIStr)) } s := upstreamURI.Query().Get("version") @@ -107,20 +107,20 @@ func init() { config, err := newSaramaConfig() if err != nil { - log.Fatal("Error creating sarama config", zap.Error(err)) + log.Panic("Error creating sarama config", zap.Error(err)) } s = upstreamURI.Query().Get("partition-num") if s == "" { partition, err := getPartitionNum(kafkaAddrs, kafkaTopic, config) if err != nil { - log.Fatal("can not get partition number", zap.String("topic", kafkaTopic), zap.Error(err)) + log.Panic("can not get partition number", zap.String("topic", kafkaTopic), zap.Error(err)) } kafkaPartitionNum = partition } else { c, err := strconv.ParseInt(s, 10, 32) if err != nil { - log.Fatal("invalid partition-num of upstream-uri") + log.Panic("invalid partition-num of upstream-uri") } kafkaPartitionNum = int32(c) } @@ -129,7 +129,7 @@ func init() { if s != "" { c, err := strconv.Atoi(s) if err != nil { - log.Fatal("invalid max-message-bytes of upstream-uri") + log.Panic("invalid max-message-bytes of upstream-uri") } log.Info("Setting max-message-bytes", zap.Int("max-message-bytes", c)) kafkaMaxMessageBytes = c @@ -139,7 +139,7 @@ func init() { if s != "" { c, err := strconv.Atoi(s) if err != nil { - log.Fatal("invalid max-batch-size of upstream-uri") + log.Panic("invalid max-batch-size of upstream-uri") } log.Info("Setting max-batch-size", zap.Int("max-batch-size", c)) kafkaMaxBatchSize = c @@ -228,24 +228,24 @@ func main() { */ config, err := newSaramaConfig() if err != nil { - log.Fatal("Error creating sarama config", zap.Error(err)) + log.Panic("Error creating sarama config", zap.Error(err)) } err = waitTopicCreated(kafkaAddrs, kafkaTopic, config) if err != nil { - log.Fatal("wait topic created failed", zap.Error(err)) + log.Panic("wait topic created failed", zap.Error(err)) } /** * Setup a new Sarama consumer group */ consumer, err := NewConsumer(context.TODO()) if err != nil { - log.Fatal("Error creating consumer", zap.Error(err)) + log.Panic("Error creating consumer", zap.Error(err)) } ctx, cancel := context.WithCancel(context.Background()) client, err := sarama.NewConsumerGroup(kafkaAddrs, kafkaGroupID, config) if err != nil { - log.Fatal("Error creating consumer group client", zap.Error(err)) + log.Panic("Error creating consumer group client", zap.Error(err)) } wg := &sync.WaitGroup{} @@ -257,7 +257,7 @@ func main() { // server-side rebalance happens, the consumer session will need to be // recreated to get the new claims if err := client.Consume(ctx, strings.Split(kafkaTopic, ","), consumer); err != nil { - log.Fatal("Error from consumer: %v", zap.Error(err)) + log.Panic("Error from consumer: %v", zap.Error(err)) } // check if context was cancelled, signaling that the consumer should stop if ctx.Err() != nil { @@ -269,7 +269,7 @@ func main() { go func() { if err := consumer.Run(ctx); err != nil { - log.Fatal("Error running consumer: %v", zap.Error(err)) + log.Panic("Error running consumer: %v", zap.Error(err)) } }() @@ -287,7 +287,7 @@ func main() { cancel() wg.Wait() if err = client.Close(); err != nil { - log.Fatal("Error closing client", zap.Error(err)) + log.Panic("Error closing client", zap.Error(err)) } } @@ -384,9 +384,9 @@ func (c *Consumer) ConsumeClaim(session sarama.ConsumerGroupSession, claim saram if sink == nil { panic("sink should initialized") } -ClaimMessages: + for message := range claim.Messages() { - log.Info("Message claimed", zap.Int32("partition", message.Partition), zap.ByteString("key", message.Key), zap.ByteString("value", message.Value)) + log.Debug("Message claimed", zap.Int32("partition", message.Partition), zap.ByteString("key", message.Key), zap.ByteString("value", message.Value)) batchDecoder, err := codec.NewJSONEventBatchDecoder(message.Key, message.Value) if err != nil { return errors.Trace(err) @@ -396,7 +396,7 @@ ClaimMessages: for { tp, hasNext, err := batchDecoder.HasNext() if err != nil { - log.Fatal("decode message key failed", zap.Error(err)) + log.Panic("decode message key failed", zap.Error(err)) } if !hasNext { break @@ -405,7 +405,7 @@ ClaimMessages: counter++ // If the message containing only one event exceeds the length limit, CDC will allow it and issue a warning. if len(message.Key)+len(message.Value) > kafkaMaxMessageBytes && counter > 1 { - log.Fatal("kafka max-messages-bytes exceeded", zap.Int("max-message-bytes", kafkaMaxMessageBytes), + log.Panic("kafka max-messages-bytes exceeded", zap.Int("max-message-bytes", kafkaMaxMessageBytes), zap.Int("receviedBytes", len(message.Key)+len(message.Value))) } @@ -413,21 +413,22 @@ ClaimMessages: case model.MqMessageTypeDDL: ddl, err := batchDecoder.NextDDLEvent() if err != nil { - log.Fatal("decode message value failed", zap.ByteString("value", message.Value)) + log.Panic("decode message value failed", zap.ByteString("value", message.Value)) } c.appendDDL(ddl) case model.MqMessageTypeRow: row, err := batchDecoder.NextRowChangedEvent() if err != nil { - log.Fatal("decode message value failed", zap.ByteString("value", message.Value)) + log.Panic("decode message value failed", zap.ByteString("value", message.Value)) } globalResolvedTs := atomic.LoadUint64(&c.globalResolvedTs) if row.CommitTs <= globalResolvedTs || row.CommitTs <= sink.resolvedTs { - log.Debug("filter fallback row", zap.ByteString("row", message.Key), + log.Debug("RowChangedEvent fallback row, ignore it", + zap.Uint64("commitTs", row.CommitTs), zap.Uint64("globalResolvedTs", globalResolvedTs), zap.Uint64("sinkResolvedTs", sink.resolvedTs), - zap.Int32("partition", partition)) - break ClaimMessages + zap.Int32("partition", partition), + zap.ByteString("row", message.Key)) } // FIXME: hack to set start-ts in row changed event, as start-ts // is not contained in TiCDC open protocol @@ -440,9 +441,8 @@ ClaimMessages: c.fakeTableIDGenerator.generateFakeTableID(row.Table.Schema, row.Table.Table, partitionID) err = sink.EmitRowChangedEvents(ctx, row) if err != nil { - log.Fatal("emit row changed event failed", zap.Error(err)) + log.Panic("emit row changed event failed", zap.Error(err)) } - log.Info("Emit RowChangedEvent", zap.Any("row", row)) lastCommitTs, ok := sink.tablesMap.Load(row.Table.TableID) if !ok || lastCommitTs.(uint64) < row.CommitTs { sink.tablesMap.Store(row.Table.TableID, row.CommitTs) @@ -450,21 +450,29 @@ ClaimMessages: case model.MqMessageTypeResolved: ts, err := batchDecoder.NextResolvedEvent() if err != nil { - log.Fatal("decode message value failed", zap.ByteString("value", message.Value)) + log.Panic("decode message value failed", zap.ByteString("value", message.Value)) } resolvedTs := atomic.LoadUint64(&sink.resolvedTs) - if resolvedTs < ts { + // `resolvedTs` should be monotonically increasing, it's allowed to receive redandunt one. + if ts < resolvedTs { + log.Panic("partition resolved ts fallback", + zap.Uint64("ts", ts), + zap.Uint64("resolvedTs", resolvedTs), + zap.Int32("partition", partition)) + } else if ts > resolvedTs { log.Debug("update sink resolved ts", zap.Uint64("ts", ts), zap.Int32("partition", partition)) atomic.StoreUint64(&sink.resolvedTs, ts) + } else { + log.Info("redundant sink resolved ts", zap.Uint64("ts", ts), zap.Int32("partition", partition)) } } session.MarkMessage(message, "") } if counter > kafkaMaxBatchSize { - log.Fatal("Open Protocol max-batch-size exceeded", zap.Int("max-batch-size", kafkaMaxBatchSize), + log.Panic("Open Protocol max-batch-size exceeded", zap.Int("max-batch-size", kafkaMaxBatchSize), zap.Int("actual-batch-size", counter)) } } @@ -479,8 +487,11 @@ func (c *Consumer) appendDDL(ddl *model.DDLEvent) { return } globalResolvedTs := atomic.LoadUint64(&c.globalResolvedTs) - if ddl.CommitTs <= globalResolvedTs { - log.Error("unexpected ddl job", zap.Uint64("ddlts", ddl.CommitTs), zap.Uint64("globalResolvedTs", globalResolvedTs)) + if ddl.CommitTs < globalResolvedTs { + log.Panic("unexpected ddl job", zap.Uint64("ddlts", ddl.CommitTs), zap.Uint64("globalResolvedTs", globalResolvedTs)) + } + if ddl.CommitTs == globalResolvedTs { + log.Warn("receive redundant ddl job", zap.Uint64("ddlts", ddl.CommitTs), zap.Uint64("globalResolvedTs", globalResolvedTs)) return } c.ddlList = append(c.ddlList, ddl) @@ -521,14 +532,15 @@ func (c *Consumer) forEachSink(fn func(sink *partitionSink) error) error { // Run runs the Consumer func (c *Consumer) Run(ctx context.Context) error { var lastGlobalResolvedTs uint64 + ticker := time.NewTicker(100 * time.Millisecond) + defer ticker.Stop() for { select { case <-ctx.Done(): return ctx.Err() - default: + case <-ticker.C: } - time.Sleep(100 * time.Millisecond) - // handle ddl + // initialize the `globalResolvedTs` as min of all partition's `ResolvedTs` globalResolvedTs := uint64(math.MaxUint64) err := c.forEachSink(func(sink *partitionSink) error { resolvedTs := atomic.LoadUint64(&sink.resolvedTs) @@ -540,6 +552,7 @@ func (c *Consumer) Run(ctx context.Context) error { if err != nil { return errors.Trace(err) } + // handle ddl todoDDL := c.getFrontDDL() if todoDDL != nil && globalResolvedTs >= todoDDL.CommitTs { // flush DMLs @@ -562,18 +575,21 @@ func (c *Consumer) Run(ctx context.Context) error { if todoDDL != nil && todoDDL.CommitTs < globalResolvedTs { globalResolvedTs = todoDDL.CommitTs } - if lastGlobalResolvedTs == globalResolvedTs { - continue + if lastGlobalResolvedTs > globalResolvedTs { + log.Panic("global ResolvedTs fallback") } - lastGlobalResolvedTs = globalResolvedTs - atomic.StoreUint64(&c.globalResolvedTs, globalResolvedTs) - log.Info("update globalResolvedTs", zap.Uint64("ts", globalResolvedTs)) - err = c.forEachSink(func(sink *partitionSink) error { - return syncFlushRowChangedEvents(ctx, sink, globalResolvedTs) - }) - if err != nil { - return errors.Trace(err) + if globalResolvedTs > lastGlobalResolvedTs { + lastGlobalResolvedTs = globalResolvedTs + atomic.StoreUint64(&c.globalResolvedTs, globalResolvedTs) + log.Info("update globalResolvedTs", zap.Uint64("ts", globalResolvedTs)) + + err = c.forEachSink(func(sink *partitionSink) error { + return syncFlushRowChangedEvents(ctx, sink, globalResolvedTs) + }) + if err != nil { + return errors.Trace(err) + } } } } diff --git a/deployments/ticdc/docker/integration-test.Dockerfile b/deployments/ticdc/docker/integration-test.Dockerfile index 03371cfa4f0..09f9c4cf87e 100644 --- a/deployments/ticdc/docker/integration-test.Dockerfile +++ b/deployments/ticdc/docker/integration-test.Dockerfile @@ -39,9 +39,11 @@ RUN yum install -y \ RUN wget http://dl.fedoraproject.org/pub/epel/epel-release-latest-7.noarch.rpm RUN yum install -y epel-release-latest-7.noarch.rpm RUN yum --enablerepo=epel install -y s3cmd -RUN wget -i -c http://dev.mysql.com/get/mysql57-community-release-el7-10.noarch.rpm -RUN yum install -y mysql57-community-release-el7-10.noarch.rpm -RUN yum install -y mysql-server +# Install mysql client. +RUN rpm -ivh https://repo.mysql.com/mysql57-community-release-el7-11.noarch.rpm +# See: https://support.cpanel.net/hc/en-us/articles/4419382481815?input_string=gpg+keys+problem+with+mysql+5.7 +RUN rpm --import https://repo.mysql.com/RPM-GPG-KEY-mysql-2022 +RUN yum install mysql-community-client.x86_64 -y # Copy go form downloader. COPY --from=downloader /usr/local/go /usr/local/go diff --git a/dm/_utils/terror_gen/errors_release.txt b/dm/_utils/terror_gen/errors_release.txt index df7b8731364..2697655c99c 100644 --- a/dm/_utils/terror_gen/errors_release.txt +++ b/dm/_utils/terror_gen/errors_release.txt @@ -182,6 +182,8 @@ ErrConfigOnlineDDLMistakeRegex,[code=20049:class=config:scope=internal:level=hig ErrOpenAPITaskConfigExist,[code=20050:class=config:scope=internal:level=low], "Message: the openapi task config for '%s' already exist, Workaround: If you want to override it, please use the overwrite flag." ErrOpenAPITaskConfigNotExist,[code=20051:class=config:scope=internal:level=low], "Message: the openapi task config for '%s' does not exist" ErrConfigCollationCompatibleNotSupport,[code=20052:class=config:scope=internal:level=medium], "Message: collation compatible %s not supported, Workaround: Please check the `collation_compatible` config in task configuration file, which can be set to `loose`/`strict`." +ErrConfigInvalidLoadMode,[code=20053:class=config:scope=internal:level=medium], "Message: invalid load mode '%s', Workaround: Please choose a valid value in ['sql', 'loader']" +ErrConfigInvalidDuplicateResolution,[code=20054:class=config:scope=internal:level=medium], "Message: invalid load on-duplicate '%s', Workaround: Please choose a valid value in ['replace', 'error', 'ignore']" ErrBinlogExtractPosition,[code=22001:class=binlog-op:scope=internal:level=high] ErrBinlogInvalidFilename,[code=22002:class=binlog-op:scope=internal:level=high], "Message: invalid binlog filename" ErrBinlogParsePosFromStr,[code=22003:class=binlog-op:scope=internal:level=high] @@ -269,7 +271,6 @@ ErrLoadUnitDuplicateTableFile,[code=34015:class=load-unit:scope=internal:level=h ErrLoadUnitGenBAList,[code=34016:class=load-unit:scope=internal:level=high], "Message: generate block allow list, Workaround: Please check the `block-allow-list` config in task configuration file." ErrLoadTaskWorkerNotMatch,[code=34017:class=functional:scope=internal:level=high], "Message: different worker in load stage, previous worker: %s, current worker: %s, Workaround: Please check if the previous worker is online." ErrLoadTaskCheckPointNotMatch,[code=34018:class=functional:scope=internal:level=high], "Message: inconsistent checkpoints between loader and target database, Workaround: If you want to redo the whole task, please check that you have not forgotten to add -remove-meta flag for start-task command." -ErrLoadBackendNotSupport,[code=34019:class=functional:scope=internal:level=high], "Message: DM do not support backend %s , Workaround: If you do not understand the configure `tidb.backend` you can just delete it." ErrSyncerUnitPanic,[code=36001:class=sync-unit:scope=internal:level=high], "Message: panic error: %v" ErrSyncUnitInvalidTableName,[code=36002:class=sync-unit:scope=internal:level=high], "Message: extract table name for DML error: %s" ErrSyncUnitTableNameQuery,[code=36003:class=sync-unit:scope=internal:level=high], "Message: table name parse error: %s" @@ -394,6 +395,7 @@ ErrMasterBoundChanging,[code=38052:class=dm-master:scope=internal:level=low], "M ErrMasterFailToImportFromV10x,[code=38053:class=dm-master:scope=internal:level=high], "Message: fail to import DM cluster from v1.0.x, Workaround: Please confirm that you have not violated any restrictions in the upgrade documentation." ErrMasterInconsistentOptimisticDDLsAndInfo,[code=38054:class=dm-master:scope=internal:level=high], "Message: inconsistent count of optimistic ddls and table infos, ddls: %d, table info: %d" ErrMasterOptimisticTableInfoBeforeNotExist,[code=38055:class=dm-master:scope=internal:level=high], "Message: table-info-before not exist in optimistic ddls: %v" +ErrMasterOptimisticDownstreamMetaNotFound,[code=38056:class=dm-master:scope=internal:level=high], "Message: downstream database config and meta for task %s not found" ErrWorkerParseFlagSet,[code=40001:class=dm-worker:scope=internal:level=medium], "Message: parse dm-worker config flag set" ErrWorkerInvalidFlag,[code=40002:class=dm-worker:scope=internal:level=medium], "Message: '%s' is an invalid flag" ErrWorkerDecodeConfigFromFile,[code=40003:class=dm-worker:scope=internal:level=medium], "Message: toml decode file, Workaround: Please check the configuration file has correct TOML format." @@ -534,6 +536,7 @@ ErrSchedulerStopRelayOnSpecified,[code=46029:class=scheduler:scope=internal:leve ErrSchedulerStartRelayOnBound,[code=46030:class=scheduler:scope=internal:level=low], "Message: the source has `start-relay` automatically for bound worker, so it can't `start-relay` with worker name now, Workaround: Please stop relay by `stop-relay` without worker name first." ErrSchedulerStopRelayOnBound,[code=46031:class=scheduler:scope=internal:level=low], "Message: the source has `start-relay` automatically for bound worker, so it can't `stop-relay` with worker name now, Workaround: Please use `stop-relay` without worker name." ErrSchedulerPauseTaskForTransferSource,[code=46032:class=scheduler:scope=internal:level=low], "Message: failed to auto pause tasks %s when transfer-source, Workaround: Please pause task by `dmctl pause-task`." +ErrSchedulerWorkerNotFree,[code=46033:class=scheduler:scope=internal:level=low], "Message: dm-worker with name %s not free" ErrCtlGRPCCreateConn,[code=48001:class=dmctl:scope=internal:level=high], "Message: can not create grpc connection, Workaround: Please check your network connection." ErrCtlInvalidTLSCfg,[code=48002:class=dmctl:scope=internal:level=medium], "Message: invalid TLS config, Workaround: Please check the `ssl-ca`, `ssl-cert` and `ssl-key` config in command line." ErrCtlLoadTLSCfg,[code=48003:class=dmctl:scope=internal:level=high], "Message: can not load tls config, Workaround: Please ensure that the tls certificate is accessible on the node currently running dmctl." diff --git a/dm/dm/common/common.go b/dm/dm/common/common.go index 4464dbb5b2b..a447c8ee373 100644 --- a/dm/dm/common/common.go +++ b/dm/dm/common/common.go @@ -78,7 +78,8 @@ var ( ShardDDLOptimismOperationKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/shardddl-optimism/operation/") // ShardDDLOptimismInitSchemaKeyAdapter is used to store the initial schema (before constructed the lock) of merged tables. // k/v: Encode(task-name, downstream-schema-name, downstream-table-name) -> table schema. - ShardDDLOptimismInitSchemaKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/shardddl-optimism/init-schema/") + // TODO: prune in etcd when upgrade + // ShardDDLOptimismInitSchemaKeyAdapter KeyAdapter = keyHexEncoderDecoder("/dm-master/shardddl-optimism/init-schema/") // ShardDDLOptimismDroppedColumnsKeyAdapter is used to store the columns that are not fully dropped // k/v: Encode(lock-id, column-name, source-id, upstream-schema-name, upstream-table-name) -> int // If we don't identify different upstream tables, we may report an error for tb2 in the following case. @@ -112,8 +113,6 @@ func keyAdapterKeysLen(s KeyAdapter) int { ShardDDLPessimismInfoKeyAdapter, ShardDDLPessimismOperationKeyAdapter, ShardDDLOptimismSourceTablesKeyAdapter, LoadTaskKeyAdapter, TaskCliArgsKeyAdapter: return 2 - case ShardDDLOptimismInitSchemaKeyAdapter: - return 3 case ShardDDLOptimismInfoKeyAdapter, ShardDDLOptimismOperationKeyAdapter: return 4 case ShardDDLOptimismDroppedColumnsKeyAdapter: diff --git a/dm/dm/common/common_test.go b/dm/dm/common/common_test.go index b2669ca5cc9..0dbebb690d3 100644 --- a/dm/dm/common/common_test.go +++ b/dm/dm/common/common_test.go @@ -16,7 +16,6 @@ package common import ( "net" "path" - "strings" "testing" . "github.com/pingcap/check" @@ -61,11 +60,6 @@ func (t *testCommon) TestKeyAdapter(c *C) { adapter: UpstreamSubTaskKeyAdapter, want: "/dm-master/upstream/subtask/6d7973716c31/74657374", }, - { - keys: []string{"test", "target_db", "target_table"}, - adapter: ShardDDLOptimismInitSchemaKeyAdapter, - want: "/dm-master/shardddl-optimism/init-schema/74657374/7461726765745f6462/7461726765745f7461626c65", - }, { keys: []string{"test", "mysql_replica_01", "target_db", "target_table"}, adapter: ShardDDLOptimismInfoKeyAdapter, @@ -108,11 +102,6 @@ func (t *testCommon) TestEncodeAsPrefix(c *C) { adapter: UpstreamSubTaskKeyAdapter, want: "/dm-master/upstream/subtask/6d7973716c31/", }, - { - keys: []string{"test", "target_db"}, - adapter: ShardDDLOptimismInitSchemaKeyAdapter, - want: "/dm-master/shardddl-optimism/init-schema/74657374/7461726765745f6462/", - }, } for _, ca := range testCases { @@ -121,18 +110,6 @@ func (t *testCommon) TestEncodeAsPrefix(c *C) { _, err := ca.adapter.Decode(encKey) c.Assert(err, NotNil) } - - keys := []string{"test", "target_db", "target_table"} - fullEncodedKey := ShardDDLOptimismInitSchemaKeyAdapter.Encode(keys...) - prefixEncodedKey := ShardDDLOptimismInitSchemaKeyAdapter.Encode(keys[:len(keys)-1]...) - c.Assert(strings.HasPrefix(fullEncodedKey, prefixEncodedKey), IsTrue) - - keys2 := []string{"test", "target_db_2", "target_table_2"} - fullEncodedKey2 := ShardDDLOptimismInitSchemaKeyAdapter.Encode(keys2...) - prefixEncodedKey2 := ShardDDLOptimismInitSchemaKeyAdapter.Encode(keys2[:len(keys2)-1]...) - - c.Assert(strings.HasPrefix(fullEncodedKey, prefixEncodedKey2), IsFalse) - c.Assert(strings.HasPrefix(fullEncodedKey2, prefixEncodedKey), IsFalse) } func (t *testCommon) TestIsErrNetClosing(c *C) { diff --git a/dm/dm/config/subtask.go b/dm/dm/config/subtask.go index e8ec8bbb964..4c63b5d2f33 100644 --- a/dm/dm/config/subtask.go +++ b/dm/dm/config/subtask.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb-tools/pkg/column-mapping" "github.com/pingcap/tidb-tools/pkg/filter" router "github.com/pingcap/tidb-tools/pkg/table-router" - lcfg "github.com/pingcap/tidb/br/pkg/lightning/config" "go.uber.org/zap" "github.com/pingcap/tiflow/dm/pkg/dumpling" @@ -174,37 +173,6 @@ func GetDBConfigForTest() DBConfig { return DBConfig{Host: "localhost", User: "root", Password: "not a real password", Port: 3306} } -// TiDBExtraConfig is the extra DB configuration only for TiDB. -type TiDBExtraConfig struct { - StatusPort int `toml:"status-port" json:"status-port" yaml:"status-port"` - PdAddr string `toml:"pd-addr" json:"pd-addr" yaml:"pd-addr"` - Backend string `toml:"backend" json:"backend" yaml:"backend"` -} - -func (db *TiDBExtraConfig) String() string { - cfg, err := json.Marshal(db) - if err != nil { - log.L().Error("fail to marshal config to json", log.ShortError(err)) - } - return string(cfg) -} - -// Toml returns TOML format representation of config. -func (db *TiDBExtraConfig) Toml() (string, error) { - var b bytes.Buffer - enc := toml.NewEncoder(&b) - if err := enc.Encode(db); err != nil { - return "", terror.ErrConfigTomlTransform.Delegate(err, "encode db config to toml") - } - return b.String(), nil -} - -// Decode loads config from file data. -func (db *TiDBExtraConfig) Decode(data string) error { - _, err := toml.Decode(data, db) - return terror.ErrConfigTomlTransform.Delegate(err, "decode db config") -} - // SubTaskConfig is the configuration for SubTask. type SubTaskConfig struct { // BurntSushi/toml seems have a bug for flag "-" @@ -256,10 +224,9 @@ type SubTaskConfig struct { RelayDir string `toml:"relay-dir" json:"relay-dir"` // UseRelay get value from dm-worker's relayEnabled - UseRelay bool `toml:"use-relay" json:"use-relay"` - From DBConfig `toml:"from" json:"from"` - To DBConfig `toml:"to" json:"to"` - TiDB TiDBExtraConfig `toml:"tidb" json:"tidb"` + UseRelay bool `toml:"use-relay" json:"use-relay"` + From DBConfig `toml:"from" json:"from"` + To DBConfig `toml:"to" json:"to"` RouteRules []*router.TableRule `toml:"route-rules" json:"route-rules"` FilterRules []*bf.BinlogEventRule `toml:"filter-rules" json:"filter-rules"` @@ -469,12 +436,12 @@ func (c *SubTaskConfig) Adjust(verifyDecryptPassword bool) error { return terror.ErrConfigInvalidChunkFileSize.Generate(c.MydumperConfig.ChunkFilesize) } - if c.TiDB.Backend != "" && c.TiDB.Backend != lcfg.BackendLocal && c.TiDB.Backend != lcfg.BackendTiDB { - return terror.ErrLoadBackendNotSupport.Generate(c.TiDB.Backend) - } if _, err := bf.NewBinlogEvent(c.CaseSensitive, c.FilterRules); err != nil { return terror.ErrConfigBinlogEventFilter.Delegate(err) } + if err := c.LoaderConfig.adjust(); err != nil { + return err + } // TODO: check every member // TODO: since we checked here, we could remove other terror like ErrSyncerUnitGenBAList @@ -558,5 +525,5 @@ func (c *SubTaskConfig) Clone() (*SubTaskConfig, error) { // NeedUseLightning returns whether need to use lightning loader. func (c *SubTaskConfig) NeedUseLightning() bool { - return (c.Mode == ModeAll || c.Mode == ModeFull) && c.TiDB.Backend != "" + return (c.Mode == ModeAll || c.Mode == ModeFull) && c.ImportMode == LoadModeSQL } diff --git a/dm/dm/config/task.go b/dm/dm/config/task.go index 6c6cedcefdb..0ce58d97d55 100644 --- a/dm/dm/config/task.go +++ b/dm/dm/config/task.go @@ -207,11 +207,35 @@ func (m *MydumperConfig) UnmarshalYAML(unmarshal func(interface{}) error) error return nil } +// LoadMode defines different mode used in load phase. +type LoadMode string + +const ( + // LoadModeSQL means write data by sql statements, uses tidb-lightning tidb backend to load data. + LoadModeSQL LoadMode = "sql" + // LoadModeLoader is the legacy sql mode, use loader to load data. this should be replaced by sql mode in new version. + LoadModeLoader = "loader" +) + +// DuplicateResolveType defines the duplication resolution when meet duplicate rows. +type DuplicateResolveType string + +const ( + // OnDuplicateReplace represents replace the old row with new data. + OnDuplicateReplace DuplicateResolveType = "replace" + // OnDuplicateError represents return an error when meet duplicate row. + OnDuplicateError = "error" + // OnDuplicateIgnore represents ignore the new data when meet duplicate row. + OnDuplicateIgnore = "ignore" +) + // LoaderConfig represents loader process unit's specific config. type LoaderConfig struct { - PoolSize int `yaml:"pool-size" toml:"pool-size" json:"pool-size"` - Dir string `yaml:"dir" toml:"dir" json:"dir"` - SQLMode string `yaml:"-" toml:"-" json:"-"` // wrote by dump unit + PoolSize int `yaml:"pool-size" toml:"pool-size" json:"pool-size"` + Dir string `yaml:"dir" toml:"dir" json:"dir"` + SQLMode string `yaml:"-" toml:"-" json:"-"` // wrote by dump unit + ImportMode LoadMode `yaml:"import-mode" toml:"import-mode" json:"import-mode"` + OnDuplicate DuplicateResolveType `yaml:"on-duplicate" toml:"on-duplicate" json:"on-duplicate"` } // DefaultLoaderConfig return default loader config for task. @@ -235,6 +259,26 @@ func (m *LoaderConfig) UnmarshalYAML(unmarshal func(interface{}) error) error { return nil } +func (m *LoaderConfig) adjust() error { + if m.ImportMode == "" { + m.ImportMode = LoadModeSQL + } + m.ImportMode = LoadMode(strings.ToLower(string(m.ImportMode))) + if m.ImportMode != LoadModeSQL && m.ImportMode != LoadModeLoader { + return terror.ErrConfigInvalidLoadMode.Generate(m.ImportMode) + } + + if m.OnDuplicate == "" { + m.OnDuplicate = OnDuplicateReplace + } + m.OnDuplicate = DuplicateResolveType(strings.ToLower(string(m.OnDuplicate))) + if m.OnDuplicate != OnDuplicateReplace && m.OnDuplicate != OnDuplicateError && m.OnDuplicate != OnDuplicateIgnore { + return terror.ErrConfigInvalidDuplicateResolution.Generate(m.OnDuplicate) + } + + return nil +} + // SyncerConfig represents syncer process unit's specific config. type SyncerConfig struct { MetaFile string `yaml:"meta-file" toml:"meta-file" json:"meta-file"` // meta filename, used only when load SubConfig directly @@ -344,9 +388,6 @@ type TaskConfig struct { // deprecated, replaced by `start-task --remove-meta` RemoveMeta bool `yaml:"remove-meta"` - // extra config when target db is TiDB - TiDB *TiDBExtraConfig `yaml:"tidb" toml:"tidb" json:"tidb"` - // task experimental configs Experimental struct { AsyncCheckpointFlush bool `yaml:"async-checkpoint-flush" toml:"async-checkpoint-flush" json:"async-checkpoint-flush"` @@ -696,7 +737,10 @@ func (c *TaskConfig) adjust() error { unusedConfigs = append(unusedConfigs, mydumper) } } - for loader := range c.Loaders { + for loader, cfg := range c.Loaders { + if err1 := cfg.adjust(); err1 != nil { + return err1 + } if globalConfigReferCount[configRefPrefixes[loaderIdx]+loader] == 0 { unusedConfigs = append(unusedConfigs, loader) } diff --git a/dm/dm/config/task_cli_args.go b/dm/dm/config/task_cli_args.go index f561b8eebd1..9f1bcedf420 100644 --- a/dm/dm/config/task_cli_args.go +++ b/dm/dm/config/task_cli_args.go @@ -15,6 +15,9 @@ package config import ( "encoding/json" + "time" + + "github.com/pingcap/tiflow/dm/pkg/terror" ) // TaskCliArgs is the task command line arguments, these arguments have higher priority than the config file and @@ -37,3 +40,16 @@ func (t *TaskCliArgs) Decode(data []byte) error { err := json.Unmarshal(data, t) return err } + +// Verify checks if all fields are legal. +func (t *TaskCliArgs) Verify() error { + if t.StartTime == "" { + return nil + } + _, err := time.Parse("2006-01-02 15:04:05", t.StartTime) + if err == nil { + return nil + } + _, err = time.Parse("2006-01-02T15:04:05", t.StartTime) + return terror.Annotate(err, "error while parse start-time, expected in the format like '2006-01-02 15:04:05'") +} diff --git a/dm/dm/config/task_cli_args_test.go b/dm/dm/config/task_cli_args_test.go index 8bdd0f52399..731ffd4866d 100644 --- a/dm/dm/config/task_cli_args_test.go +++ b/dm/dm/config/task_cli_args_test.go @@ -43,3 +43,14 @@ func (t *testConfig) TestTaskCliArgsDowngrade(c *C) { c.Assert(afterDowngrade.Decode([]byte(data)), IsNil) c.Assert(afterDowngrade.StartTime, Equals, "123") } + +func (t *testConfig) TestTaskCliArgsVerify(c *C) { + empty := TaskCliArgs{} + c.Assert(empty.Verify(), IsNil) + rightStartTime := TaskCliArgs{StartTime: "2006-01-02T15:04:05"} + c.Assert(rightStartTime.Verify(), IsNil) + rightStartTime = TaskCliArgs{StartTime: "2006-01-02 15:04:05"} + c.Assert(rightStartTime.Verify(), IsNil) + wrongStartTime := TaskCliArgs{StartTime: "15:04:05"} + c.Assert(wrongStartTime.Verify(), NotNil) +} diff --git a/dm/dm/config/task_converters.go b/dm/dm/config/task_converters.go index 7d0eb289be7..99c3c8ef7a9 100644 --- a/dm/dm/config/task_converters.go +++ b/dm/dm/config/task_converters.go @@ -99,9 +99,6 @@ func TaskConfigToSubTaskConfigs(c *TaskConfig, sources map[string]DBConfig) ([]* if err := cfg.Adjust(true); err != nil { return nil, terror.Annotatef(err, "source %s", inst.SourceID) } - if c.TiDB != nil { - cfg.TiDB = *c.TiDB - } cfgs[i] = cfg } if c.EnableHeartbeat { diff --git a/dm/dm/config/task_test.go b/dm/dm/config/task_test.go index fec4f7a078e..71571d8c455 100644 --- a/dm/dm/config/task_test.go +++ b/dm/dm/config/task_test.go @@ -690,8 +690,10 @@ func (t *testConfig) TestGenAndFromSubTaskConfigs(c *C) { ExtraArgs: "--escape-backslash", }, LoaderConfig: LoaderConfig{ - PoolSize: 32, - Dir: "./dumpped_data", + PoolSize: 32, + Dir: "./dumpped_data", + ImportMode: LoadModeSQL, + OnDuplicate: OnDuplicateReplace, }, SyncerConfig: SyncerConfig{ WorkerCount: 32, @@ -1029,7 +1031,7 @@ func (t *testConfig) TestTaskConfigForDowngrade(c *C) { // make sure all new field were added cfgReflect := reflect.Indirect(reflect.ValueOf(cfg)) cfgForDowngradeReflect := reflect.Indirect(reflect.ValueOf(cfgForDowngrade)) - c.Assert(cfgReflect.NumField(), Equals, cfgForDowngradeReflect.NumField()+4) // without flag, tidb, collation_compatible and experimental + c.Assert(cfgReflect.NumField(), Equals, cfgForDowngradeReflect.NumField()+3) // without flag, collation_compatible and experimental // make sure all field were copied cfgForClone := &TaskConfigForDowngrade{} diff --git a/dm/dm/ctl/common/config.go b/dm/dm/ctl/common/config.go index a98cb59ed44..6d73fbfdf4c 100644 --- a/dm/dm/ctl/common/config.go +++ b/dm/dm/ctl/common/config.go @@ -21,7 +21,11 @@ import ( "strings" "time" + "github.com/google/shlex" + "go.uber.org/zap" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/utils" "github.com/BurntSushi/toml" @@ -215,3 +219,13 @@ func validateAddr(addr string) error { } return nil } + +// SplitArgsRespectQuote splits args by space, but won't split space inside single or double quotes. +func SplitArgsRespectQuote(line string) []string { + ret, err := shlex.Split(line) + if err != nil { + log.L().Error("split args error", zap.Error(err)) + return []string{line} + } + return ret +} diff --git a/dm/dm/ctl/common/config_test.go b/dm/dm/ctl/common/config_test.go new file mode 100644 index 00000000000..08a794804e9 --- /dev/null +++ b/dm/dm/ctl/common/config_test.go @@ -0,0 +1,53 @@ +// Copyright 2022 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 common + +import ( + "testing" + + . "github.com/pingcap/check" +) + +func TestConfig(t *testing.T) { + TestingT(t) +} + +var _ = Suite(&testConfigSuite{}) + +type testConfigSuite struct{} + +func (t *testConfigSuite) TestInteractiveQuotes(c *C) { + cases := []struct { + input string + expected []string + }{ + {`123`, []string{`123`}}, + {`"123"`, []string{`123`}}, + {`'123'`, []string{`123`}}, + {`123 456`, []string{`123`, `456`}}, + {`'123 456'`, []string{`123 456`}}, + {`"123 456"`, []string{`123 456`}}, + {`"123 456" 789`, []string{`123 456`, `789`}}, + {`0 '123"456 789'`, []string{`0`, `123"456 789`}}, + {`0'123"456 789'`, []string{`0123"456 789`}}, + {`"123""456" 7 "89"`, []string{`123456`, `7`, `89`}}, + // return original string when failed to split + {`123"456`, []string{`123"456`}}, + } + + for _, ca := range cases { + got := SplitArgsRespectQuote(ca.input) + c.Assert(got, DeepEquals, ca.expected) + } +} diff --git a/dm/dm/ctl/ctl.go b/dm/dm/ctl/ctl.go index 94f40e1cf41..aad9b046b9f 100644 --- a/dm/dm/ctl/ctl.go +++ b/dm/dm/ctl/ctl.go @@ -154,7 +154,7 @@ func loop() error { continue } - args := strings.Fields(line) + args := common.SplitArgsRespectQuote(line) c, err := Start(args) if err != nil { fmt.Println("fail to run:", args) diff --git a/dm/dm/ctl/master/check_task.go b/dm/dm/ctl/master/check_task.go index b3536f66c37..52c9b2c78f2 100644 --- a/dm/dm/ctl/master/check_task.go +++ b/dm/dm/ctl/master/check_task.go @@ -34,6 +34,7 @@ func NewCheckTaskCmd() *cobra.Command { } cmd.Flags().Int64P("error", "e", common.DefaultErrorCnt, "max count of errors to display") cmd.Flags().Int64P("warn", "w", common.DefaultWarnCnt, "max count of warns to display") + cmd.Flags().String("start-time", "", "specify the start time of binlog replication, e.g. '2021-10-21 00:01:00' or 2021-10-21T00:01:00") return cmd } @@ -57,6 +58,10 @@ func checkTaskFunc(cmd *cobra.Command, _ []string) error { if err != nil { return err } + startTime, err := cmd.Flags().GetString("start-time") + if err != nil { + return err + } ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -67,9 +72,10 @@ func checkTaskFunc(cmd *cobra.Command, _ []string) error { ctx, "CheckTask", &pb.CheckTaskRequest{ - Task: string(content), - ErrCnt: errCnt, - WarnCnt: warnCnt, + Task: string(content), + ErrCnt: errCnt, + WarnCnt: warnCnt, + StartTime: startTime, }, &resp, ) diff --git a/dm/dm/ctl/master/operate_source.go b/dm/dm/ctl/master/operate_source.go index c50e405217d..bba20d1c7b8 100644 --- a/dm/dm/ctl/master/operate_source.go +++ b/dm/dm/ctl/master/operate_source.go @@ -32,11 +32,12 @@ import ( // NewOperateSourceCmd creates a OperateSource command. func NewOperateSourceCmd() *cobra.Command { cmd := &cobra.Command{ - Use: "operate-source [config-file ...] [--print-sample-config]", + Use: "operate-source [config-file ...] [-w worker] [--print-sample-config]", Short: "`create`/`update`/`stop`/`show` upstream MySQL/MariaDB source", RunE: operateSourceFunc, } cmd.Flags().BoolP("print-sample-config", "p", false, "print sample config file of source") + cmd.Flags().StringP("worker", "w", "", "specify bound worker for created source") return cmd } @@ -85,6 +86,20 @@ func operateSourceFunc(cmd *cobra.Command, _ []string) error { return errors.New("please check output to see error") } + var specifyWorker string + if op == pb.SourceOp_StartSource { + specifyWorker, err = cmd.Flags().GetString("worker") + if err != nil { + common.PrintLinesf("error in parse `--worker`") + return err + } + if specifyWorker != "" { + if len(cmd.Flags().Args()) > 2 { + common.PrintLinesf("operate-source create can't create multiple sources when specify worker") + } + } + } + contents := make([]string, 0, len(cmd.Flags().Args())-1) sourceID := make([]string, 0, len(cmd.Flags().Args())-1) sources, err := common.GetSourceArgs(cmd) @@ -132,9 +147,10 @@ func operateSourceFunc(cmd *cobra.Command, _ []string) error { ctx, "OperateSource", &pb.OperateSourceRequest{ - Config: contents, - Op: op, - SourceID: sourceID, + Config: contents, + Op: op, + SourceID: sourceID, + WorkerName: specifyWorker, }, &resp, ) diff --git a/dm/dm/ctl/master/start_task.go b/dm/dm/ctl/master/start_task.go index c748c7419e5..6d70bd41999 100644 --- a/dm/dm/ctl/master/start_task.go +++ b/dm/dm/ctl/master/start_task.go @@ -37,6 +37,7 @@ func NewStartTaskCmd() *cobra.Command { RunE: startTaskFunc, } cmd.Flags().BoolP("remove-meta", "", false, "whether to remove task's meta data") + cmd.Flags().String("start-time", "", "specify the start time of binlog replication, e.g. '2021-10-21 00:01:00' or 2021-10-21T00:01:00") return cmd } @@ -76,6 +77,11 @@ func startTaskFunc(cmd *cobra.Command, _ []string) error { common.PrintLinesf("error in parse `--remove-meta`") return err } + startTime, err := cmd.Flags().GetString("start-time") + if err != nil { + common.PrintLinesf("error in parse `--start-time`") + return err + } ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -89,6 +95,7 @@ func startTaskFunc(cmd *cobra.Command, _ []string) error { Task: string(content), Sources: sources, RemoveMeta: removeMeta, + StartTime: startTime, }, &resp, ) diff --git a/dm/dm/dm-ansible/scripts/DM-Monitor-Professional.json b/dm/dm/dm-ansible/scripts/DM-Monitor-Professional.json index a7c0d6006cf..d05a6730bc0 100644 --- a/dm/dm/dm-ansible/scripts/DM-Monitor-Professional.json +++ b/dm/dm/dm-ansible/scripts/DM-Monitor-Professional.json @@ -3059,7 +3059,7 @@ "steppedLine": false, "targets": [ { - "expr": "dm_syncer_binlog_file{source_id=~\"$source\", task=~\"$task\", node=\"master\"} - ON( task,source_id) dm_syncer_binlog_file{source_id=~\"$source\", task=~\"$task\", node=\"syncer\"}", + "expr": "dm_syncer_binlog_file{source_id=~\"$source\", task=~\"$task\", node=\"master\"} - ON(task,source_id) dm_syncer_binlog_file{source_id=~\"$source\", task=~\"$task\", node=\"syncer\"}", "format": "time_series", "hide": false, "interval": "$interval", @@ -3716,12 +3716,12 @@ "steppedLine": false, "targets": [ { - "expr": "dm_relay_binlog_file{instance=~\"$instance\", node=\"relay\"} - ON(instance, task) dm_syncer_binlog_file{instance=~\"$instance\", task=~\"$task\", node=\"syncer\"}", + "expr": "dm_relay_binlog_file{instance=~\"$instance\", node=\"relay\"} - ON(instance) group_right dm_syncer_binlog_file{instance=~\"$instance\", task=~\"$task\", node=\"syncer\"}", "format": "time_series", "hide": false, "interval": "$interval", "intervalFactor": 2, - "legendFormat": "{{task}} - {{instance}}", + "legendFormat": "{{instance}} - {{task}}", "refId": "B" } ], @@ -5088,7 +5088,7 @@ "steppedLine": false, "targets": [ { - "expr": "dm_relay_binlog_file{instance=~\"$instance\", node=\"master\"} - ON(instance, job) dm_relay_binlog_file{instance=~\"$instance\", node=\"relay\"}", + "expr": "dm_relay_binlog_file{instance=~\"$instance\", node=\"master\"} - ON(instance) dm_relay_binlog_file{instance=~\"$instance\", node=\"relay\"}", "format": "time_series", "hide": false, "interval": "$interval", diff --git a/dm/dm/dm-ansible/scripts/DM-Monitor-Standard.json b/dm/dm/dm-ansible/scripts/DM-Monitor-Standard.json index 7b8bf19e037..e29a8005e40 100644 --- a/dm/dm/dm-ansible/scripts/DM-Monitor-Standard.json +++ b/dm/dm/dm-ansible/scripts/DM-Monitor-Standard.json @@ -450,7 +450,7 @@ "targets": [ { "exemplar": true, - "expr": "histogram_quantile(0.90, sum(rate(dm_syncer_replication_lag_bucket{instance=~\"$instance\",task=~\"$task\"}[$interval])) by (le,instance,task))", + "expr": "histogram_quantile(0.90, sum(rate(dm_syncer_replication_lag_bucket{source_id=~\"$source\",task=~\"$task\"}[$interval])) by (le,task,source_id))", "hide": false, "interval": "", "legendFormat": "{{task}} - {{source_id}} - 90", @@ -458,7 +458,7 @@ }, { "exemplar": true, - "expr": "histogram_quantile(0.95,\nsum(rate(dm_syncer_replication_lag_bucket{instance=~\"$instance\",task=~\"$task\"}[$interval])) by (le,instance,task))", + "expr": "histogram_quantile(0.95,\nsum(rate(dm_syncer_replication_lag_bucket{source_id=~\"$source\",task=~\"$task\"}[$interval])) by (le,task,source_id))", "hide": false, "interval": "", "legendFormat": "{{task}} - {{source_id}} - 95", @@ -466,7 +466,7 @@ }, { "exemplar": true, - "expr": "histogram_quantile(0.99,\nsum(rate(dm_syncer_replication_lag_bucket{instance=~\"$instance\",task=~\"$task\"}[$interval])) by (le,instance,task))", + "expr": "histogram_quantile(0.99,\nsum(rate(dm_syncer_replication_lag_bucket{source_id=~\"$source\",task=~\"$task\"}[$interval])) by (le,task,source_id))", "hide": false, "interval": "", "legendFormat": "{{task}} - {{source_id}} - 99", diff --git a/dm/dm/master/scheduler/scheduler.go b/dm/dm/master/scheduler/scheduler.go index b2062ac7ece..c0784c57392 100644 --- a/dm/dm/master/scheduler/scheduler.go +++ b/dm/dm/master/scheduler/scheduler.go @@ -220,27 +220,27 @@ func (s *Scheduler) Start(pCtx context.Context, etcdCli *clientv3.Client) (err e s.reset() // reset previous status. // recover previous status from etcd. - err = s.recoverSources(etcdCli) + err = s.recoverSources() if err != nil { return err } - err = s.recoverSubTasks(etcdCli) + err = s.recoverSubTasks() if err != nil { return err } - err = s.recoverRelayConfigs(etcdCli) + err = s.recoverRelayConfigs() if err != nil { return err } var loadTaskRev int64 - loadTaskRev, err = s.recoverLoadTasks(etcdCli, false) + loadTaskRev, err = s.recoverLoadTasks(false) if err != nil { return err } var rev int64 - rev, err = s.recoverWorkersBounds(etcdCli) + rev, err = s.recoverWorkersBounds() if err != nil { return err } @@ -266,7 +266,7 @@ func (s *Scheduler) Start(pCtx context.Context, etcdCli *clientv3.Client) (err e // starting to observe status of DM-worker instances. // TODO: handle fatal error from observeWorkerEvent //nolint:errcheck - s.observeWorkerEvent(ctx, etcdCli, rev1) + s.observeWorkerEvent(ctx, rev1) }(rev) s.wg.Add(1) @@ -275,7 +275,7 @@ func (s *Scheduler) Start(pCtx context.Context, etcdCli *clientv3.Client) (err e // starting to observe load task. // TODO: handle fatal error from observeLoadTask //nolint:errcheck - s.observeLoadTask(ctx, etcdCli, rev1) + s.observeLoadTask(ctx, rev1) }(loadTaskRev) s.started.Store(true) // started now @@ -317,7 +317,7 @@ func (s *Scheduler) CloseAllWorkers() { } } -// AddSourceCfg adds the upstream source config to the cluster. +// AddSourceCfg adds the upstream source config to the cluster, and try to bound source to worker // NOTE: please verify the config before call this. func (s *Scheduler) AddSourceCfg(cfg *config.SourceConfig) error { s.mu.Lock() @@ -327,11 +327,49 @@ func (s *Scheduler) AddSourceCfg(cfg *config.SourceConfig) error { return terror.ErrSchedulerNotStarted.Generate() } + err := s.addSource(cfg) + if err != nil { + return err + } + + // try to bound it to a Free worker. + _, err = s.tryBoundForSource(cfg.SourceID) + return err +} + +// AddSourceCfgWithWorker adds the upstream source config to the cluster, and try to bound source to specify worker +// NOTE: please verify the config before call this. +func (s *Scheduler) AddSourceCfgWithWorker(cfg *config.SourceConfig, workerName string) error { + s.mu.Lock() + defer s.mu.Unlock() + + if !s.started.Load() { + return terror.ErrSchedulerNotStarted.Generate() + } + + // check whether worker exists. + w, ok := s.workers[workerName] + if !ok { + return terror.ErrSchedulerWorkerNotExist.Generate(workerName) + } + + if w.stage != WorkerFree { + return terror.ErrSchedulerWorkerNotFree.Generate(workerName) + } + + if err := s.addSource(cfg); err != nil { + return err + } + + return s.boundSourceToWorker(cfg.SourceID, w) +} + +// addSource adds the upstream source config to the cluster. +func (s *Scheduler) addSource(cfg *config.SourceConfig) error { // 1. check whether exists. if _, ok := s.sourceCfgs[cfg.SourceID]; ok { return terror.ErrSchedulerSourceCfgExist.Generate(cfg.SourceID) } - // 2. put the config into etcd. _, err := ha.PutSourceCfg(s.etcdCli, cfg) if err != nil { @@ -341,10 +379,7 @@ func (s *Scheduler) AddSourceCfg(cfg *config.SourceConfig) error { // 3. record the config in the scheduler. s.sourceCfgs[cfg.SourceID] = cfg s.unbounds[cfg.SourceID] = struct{}{} - - // 4. try to bound it to a Free worker. - _, err = s.tryBoundForSource(cfg.SourceID) - return err + return nil } // UpdateSourceCfg update the upstream source config to the cluster. @@ -985,6 +1020,19 @@ func (s *Scheduler) getSubTaskCfgByTaskSource(task, source string) *config.SubTa return &clone } +// GetDownstreamMetaByTask gets downstream db config and meta config by task name. +func (s *Scheduler) GetDownstreamMetaByTask(task string) (*config.DBConfig, string) { + v, ok := s.subTaskCfgs.Load(task) + if !ok { + return nil, "" + } + cfgM := v.(map[string]config.SubTaskConfig) + for _, cfg := range cfgM { + return cfg.To.Clone(), cfg.MetaSchema + } + return nil, "" +} + // GetSubTaskCfgsByTask gets subtask configs' map by task name. func (s *Scheduler) GetSubTaskCfgsByTask(task string) map[string]*config.SubTaskConfig { v, ok := s.subTaskCfgs.Load(task) @@ -1578,14 +1626,14 @@ func (s *Scheduler) Started() bool { } // recoverSourceCfgs recovers history source configs and expectant relay stages from etcd. -func (s *Scheduler) recoverSources(cli *clientv3.Client) error { +func (s *Scheduler) recoverSources() error { // get all source configs. - cfgM, _, err := ha.GetSourceCfg(cli, "", 0) + cfgM, _, err := ha.GetSourceCfg(s.etcdCli, "", 0) if err != nil { return err } // get all relay stages. - stageM, _, err := ha.GetAllRelayStage(cli) + stageM, _, err := ha.GetAllRelayStage(s.etcdCli) if err != nil { return err } @@ -1602,14 +1650,14 @@ func (s *Scheduler) recoverSources(cli *clientv3.Client) error { } // recoverSubTasks recovers history subtask configs and expectant subtask stages from etcd. -func (s *Scheduler) recoverSubTasks(cli *clientv3.Client) error { +func (s *Scheduler) recoverSubTasks() error { // get all subtask configs. - cfgMM, _, err := ha.GetAllSubTaskCfg(cli) + cfgMM, _, err := ha.GetAllSubTaskCfg(s.etcdCli) if err != nil { return err } // get all subtask stages. - stageMM, _, err := ha.GetAllSubTaskStage(cli) + stageMM, _, err := ha.GetAllSubTaskStage(s.etcdCli) if err != nil { return err } @@ -1637,8 +1685,8 @@ func (s *Scheduler) recoverSubTasks(cli *clientv3.Client) error { // This function also removes conflicting relay schedule types, which means if a source has both `enable-relay` and // (source, worker) relay config, we remove the latter. // should be called after recoverSources. -func (s *Scheduler) recoverRelayConfigs(cli *clientv3.Client) error { - relayWorkers, _, err := ha.GetAllRelayConfig(cli) +func (s *Scheduler) recoverRelayConfigs() error { + relayWorkers, _, err := ha.GetAllRelayConfig(s.etcdCli) if err != nil { return err } @@ -1651,7 +1699,7 @@ func (s *Scheduler) recoverRelayConfigs(cli *clientv3.Client) error { } if sourceCfg.EnableRelay { // current etcd max-txn-op is 2048 - _, err2 := ha.DeleteRelayConfig(cli, utils.SetToSlice(workers)...) + _, err2 := ha.DeleteRelayConfig(s.etcdCli, utils.SetToSlice(workers)...) if err2 != nil { return err2 } @@ -1664,12 +1712,12 @@ func (s *Scheduler) recoverRelayConfigs(cli *clientv3.Client) error { } // recoverLoadTasks recovers history load workers from etcd. -func (s *Scheduler) recoverLoadTasks(cli *clientv3.Client, needLock bool) (int64, error) { +func (s *Scheduler) recoverLoadTasks(needLock bool) (int64, error) { if needLock { s.mu.Lock() defer s.mu.Unlock() } - loadTasks, rev, err := ha.GetAllLoadTask(cli) + loadTasks, rev, err := ha.GetAllLoadTask(s.etcdCli) if err != nil { return 0, err } @@ -1680,11 +1728,11 @@ func (s *Scheduler) recoverLoadTasks(cli *clientv3.Client, needLock bool) (int64 // recoverWorkersBounds recovers history DM-worker info and status from etcd. // and it also recovers the bound/unbound relationship. -func (s *Scheduler) recoverWorkersBounds(cli *clientv3.Client) (int64, error) { +func (s *Scheduler) recoverWorkersBounds() (int64, error) { // 1. get all history base info. // it should no new DM-worker registered between this call and the below `GetKeepAliveWorkers`, // because no DM-master leader are handling DM-worker register requests. - wim, _, err := ha.GetAllWorkerInfo(cli) + wim, _, err := ha.GetAllWorkerInfo(s.etcdCli) if err != nil { return 0, err } @@ -1692,18 +1740,18 @@ func (s *Scheduler) recoverWorkersBounds(cli *clientv3.Client) (int64, error) { // 2. get all history bound relationships. // it should no new bound relationship added between this call and the below `GetKeepAliveWorkers`, // because no DM-master leader are doing the scheduler. - sbm, _, err := ha.GetSourceBound(cli, "") + sbm, _, err := ha.GetSourceBound(s.etcdCli, "") if err != nil { return 0, err } - lastSourceBoundM, _, err := ha.GetLastSourceBounds(cli) + lastSourceBoundM, _, err := ha.GetLastSourceBounds(s.etcdCli) if err != nil { return 0, err } s.lastBound = lastSourceBoundM // 3. get all history offline status. - kam, rev, err := ha.GetKeepAliveWorkers(cli) + kam, rev, err := ha.GetKeepAliveWorkers(s.etcdCli) if err != nil { return 0, err } @@ -1767,7 +1815,7 @@ func (s *Scheduler) recoverWorkersBounds(cli *clientv3.Client) (int64, error) { for name := range sbm { invalidSourceBounds = append(invalidSourceBounds, name) } - _, err = ha.DeleteSourceBound(cli, invalidSourceBounds...) + _, err = ha.DeleteSourceBound(s.etcdCli, invalidSourceBounds...) if err != nil { return 0, err } @@ -1775,7 +1823,7 @@ func (s *Scheduler) recoverWorkersBounds(cli *clientv3.Client) (int64, error) { // 6. put trigger source bounds info to etcd to order dm-workers to start source if len(boundsToTrigger) > 0 { - _, err = ha.PutSourceBound(cli, boundsToTrigger...) + _, err = ha.PutSourceBound(s.etcdCli, boundsToTrigger...) if err != nil { return 0, err } @@ -1791,12 +1839,12 @@ func (s *Scheduler) recoverWorkersBounds(cli *clientv3.Client) (int64, error) { return rev, nil } -func (s *Scheduler) resetWorkerEv(cli *clientv3.Client) (int64, error) { +func (s *Scheduler) resetWorkerEv() (int64, error) { s.mu.Lock() defer s.mu.Unlock() rwm := s.workers - kam, rev, err := ha.GetKeepAliveWorkers(cli) + kam, rev, err := ha.GetKeepAliveWorkers(s.etcdCli) if err != nil { return 0, err } @@ -1856,7 +1904,7 @@ func (s *Scheduler) handleWorkerEv(ctx context.Context, evCh <-chan ha.WorkerEve } // nolint:dupl -func (s *Scheduler) observeWorkerEvent(ctx context.Context, etcdCli *clientv3.Client, rev int64) error { +func (s *Scheduler) observeWorkerEvent(ctx context.Context, rev int64) error { var wg sync.WaitGroup for { workerEvCh := make(chan ha.WorkerEvent, 10) @@ -1870,7 +1918,7 @@ func (s *Scheduler) observeWorkerEvent(ctx context.Context, etcdCli *clientv3.Cl close(workerErrCh) wg.Done() }() - ha.WatchWorkerEvent(ctx1, etcdCli, rev+1, workerEvCh, workerErrCh) + ha.WatchWorkerEvent(ctx1, s.etcdCli, rev+1, workerEvCh, workerErrCh) }() err := s.handleWorkerEv(ctx1, workerEvCh, workerErrCh) cancel1() @@ -1884,7 +1932,7 @@ func (s *Scheduler) observeWorkerEvent(ctx context.Context, etcdCli *clientv3.Cl case <-ctx.Done(): return nil case <-time.After(500 * time.Millisecond): - rev, err = s.resetWorkerEv(etcdCli) + rev, err = s.resetWorkerEv() if err != nil { log.L().Error("resetWorkerEv is failed, will retry later", zap.Error(err), zap.Int("retryNum", retryNum)) } @@ -2311,7 +2359,7 @@ func (s *Scheduler) SetWorkerClientForTest(name string, mockCli workerrpc.Client } // nolint:dupl -func (s *Scheduler) observeLoadTask(ctx context.Context, etcdCli *clientv3.Client, rev int64) error { +func (s *Scheduler) observeLoadTask(ctx context.Context, rev int64) error { var wg sync.WaitGroup for { loadTaskCh := make(chan ha.LoadTask, 10) @@ -2325,7 +2373,7 @@ func (s *Scheduler) observeLoadTask(ctx context.Context, etcdCli *clientv3.Clien close(loadTaskErrCh) wg.Done() }() - ha.WatchLoadTask(ctx1, etcdCli, rev+1, loadTaskCh, loadTaskErrCh) + ha.WatchLoadTask(ctx1, s.etcdCli, rev+1, loadTaskCh, loadTaskErrCh) }() err := s.handleLoadTask(ctx1, loadTaskCh, loadTaskErrCh) cancel1() @@ -2339,7 +2387,7 @@ func (s *Scheduler) observeLoadTask(ctx context.Context, etcdCli *clientv3.Clien case <-ctx.Done(): return nil case <-time.After(500 * time.Millisecond): - rev, err = s.recoverLoadTasks(etcdCli, true) + rev, err = s.recoverLoadTasks(true) if err != nil { log.L().Error("resetLoadTask is failed, will retry later", zap.Error(err), zap.Int("retryNum", retryNum)) } diff --git a/dm/dm/master/scheduler/scheduler_test.go b/dm/dm/master/scheduler/scheduler_test.go index cc22cbd43c6..e9eb1627e6d 100644 --- a/dm/dm/master/scheduler/scheduler_test.go +++ b/dm/dm/master/scheduler/scheduler_test.go @@ -135,6 +135,7 @@ func (t *testScheduler) testSchedulerProgress(c *C, restart int) { // not started scheduler can't do anything. c.Assert(terror.ErrSchedulerNotStarted.Equal(s.AddSourceCfg(sourceCfg1)), IsTrue) + c.Assert(terror.ErrSchedulerNotStarted.Equal(s.AddSourceCfgWithWorker(sourceCfg1, workerName1)), IsTrue) c.Assert(terror.ErrSchedulerNotStarted.Equal(s.UpdateSourceCfg(sourceCfg1)), IsTrue) c.Assert(terror.ErrSchedulerNotStarted.Equal(s.RemoveSourceCfg(sourceID1)), IsTrue) c.Assert(terror.ErrSchedulerNotStarted.Equal(s.AddSubTasks(false, subtaskCfg1)), IsTrue) @@ -253,12 +254,15 @@ func (t *testScheduler) testSchedulerProgress(c *C, restart int) { c.Assert(s.AddSubTasks(false), IsNil) // can call without configs, return without error, but take no effect. t.subTaskCfgNotExist(c, s, taskName1, sourceID1) t.subTaskStageMatch(c, s, taskName1, sourceID1, pb.Stage_InvalidStage) + t.downstreamMetaNotExist(c, s, taskName1) // start the task. c.Assert(s.AddSubTasks(false, subtaskCfg1), IsNil) c.Assert(terror.ErrSchedulerSubTaskExist.Equal(s.AddSubTasks(false, subtaskCfg1)), IsTrue) // add again. // subtask config and stage exist. t.subTaskCfgExist(c, s, subtaskCfg1) t.subTaskStageMatch(c, s, taskName1, sourceID1, pb.Stage_Running) + t.downstreamMetaExist(c, s, taskName1, subtaskCfg1.To, subtaskCfg1.MetaSchema) + t.downstreamMetaNotExist(c, s, taskName2) // update source config when task already started will failed c.Assert(terror.ErrSchedulerSourceOpTaskExist.Equal(s.UpdateSourceCfg(sourceCfg1)), IsTrue) @@ -406,7 +410,7 @@ func (t *testScheduler) testSchedulerProgress(c *C, restart int) { t.relayStageMatch(c, s, sourceID2, pb.Stage_Running) rebuildScheduler(ctx) - // CASE 4.4: start a task with two sources. + // CASE 4.4.1: start a task with two sources. // can't add more than one tasks at a time now. c.Assert(terror.ErrSchedulerMultiTask.Equal(s.AddSubTasks(false, subtaskCfg1, subtaskCfg21)), IsTrue) // task2' config and stage not exists before. @@ -423,7 +427,7 @@ func (t *testScheduler) testSchedulerProgress(c *C, restart int) { t.subTaskStageMatch(c, s, taskName2, sourceID2, pb.Stage_Running) rebuildScheduler(ctx) - // CASE 4.4.1 fail to stop any task. + // CASE 4.4.2 fail to stop any task. // can call without tasks or sources, return without error, but take no effect. c.Assert(s.RemoveSubTasks("", sourceID1), IsNil) c.Assert(s.RemoveSubTasks(taskName1), IsNil) @@ -481,6 +485,38 @@ func (t *testScheduler) testSchedulerProgress(c *C, restart int) { t.relayStageMatch(c, s, sourceID2, pb.Stage_InvalidStage) rebuildScheduler(ctx) + // CASE 4.7.1: add source2 with specify worker1 + // source2 not exist, worker1 is bound + t.sourceCfgNotExist(c, s, sourceID2) + t.workerBound(c, s, ha.NewSourceBound(sourceID1, workerName1)) + c.Assert(terror.ErrSchedulerWorkerNotFree.Equal(s.AddSourceCfgWithWorker(&sourceCfg2, workerName1)), IsTrue) + // source2 is not created because expected worker1 is already bound + t.sourceCfgNotExist(c, s, sourceID2) + rebuildScheduler(ctx) + + // CASE 4.7.2: add source2 with specify worker2 + // source2 not exist, worker2 should be free + t.sourceCfgNotExist(c, s, sourceID2) + t.workerFree(c, s, workerName2) + c.Assert(s.AddSourceCfgWithWorker(&sourceCfg2, workerName2), IsNil) + t.workerBound(c, s, ha.NewSourceBound(sourceID2, workerName2)) + t.sourceBounds(c, s, []string{sourceID1, sourceID2}, []string{}) + c.Assert(s.StartRelay(sourceID2, []string{workerName2}), IsNil) + t.relayStageMatch(c, s, sourceID2, pb.Stage_Running) + rebuildScheduler(ctx) + + // CASE 4.7.3: remove source2 again. + c.Assert(s.StopRelay(sourceID2, []string{workerName2}), IsNil) + c.Assert(s.RemoveSourceCfg(sourceID2), IsNil) + c.Assert(terror.ErrSchedulerSourceCfgNotExist.Equal(s.RemoveSourceCfg(sourceID2)), IsTrue) // already removed. + // source2 removed. + t.sourceCfgNotExist(c, s, sourceID2) + // worker2 become Free now. + t.workerFree(c, s, workerName2) + t.sourceBounds(c, s, []string{sourceID1}, []string{}) + t.relayStageMatch(c, s, sourceID2, pb.Stage_InvalidStage) + rebuildScheduler(ctx) + // CASE 4.8: worker1 become offline. // before shutdown, worker1 bound source t.workerBound(c, s, ha.NewSourceBound(sourceID1, workerName1)) @@ -596,6 +632,19 @@ func (t *testScheduler) subTaskCfgExist(c *C, s *Scheduler, expectCfg config.Sub c.Assert(cfgM[expectCfg.Name], DeepEquals, expectCfg) } +func (t *testScheduler) downstreamMetaNotExist(c *C, s *Scheduler, task string) { + dbConfig, metaConfig := s.GetDownstreamMetaByTask(task) + c.Assert(dbConfig, IsNil) + c.Assert(metaConfig, Equals, "") +} + +func (t *testScheduler) downstreamMetaExist(c *C, s *Scheduler, task string, expectDBCfg config.DBConfig, expectMetaConfig string) { + dbConfig, metaConfig := s.GetDownstreamMetaByTask(task) + c.Assert(dbConfig, NotNil) + c.Assert(dbConfig, DeepEquals, &expectDBCfg) + c.Assert(metaConfig, Equals, expectMetaConfig) +} + func (t *testScheduler) workerNotExist(c *C, s *Scheduler, worker string) { c.Assert(s.GetWorkerByName(worker), IsNil) wm, _, err := ha.GetAllWorkerInfo(etcdTestCli) @@ -986,7 +1035,7 @@ func (t *testScheduler) TestWatchWorkerEventEtcdCompact(c *C) { wg.Add(1) go func() { defer wg.Done() - c.Assert(s.observeWorkerEvent(ctx2, etcdTestCli, startRev), IsNil) + c.Assert(s.observeWorkerEvent(ctx2, startRev), IsNil) }() // step 5.3: wait for scheduler to restart handleWorkerEvent, then start a new worker time.Sleep(time.Second) @@ -1008,7 +1057,7 @@ func (t *testScheduler) TestWatchWorkerEventEtcdCompact(c *C) { wg.Add(1) go func() { defer wg.Done() - c.Assert(s.observeWorkerEvent(ctx3, etcdTestCli, startRev), IsNil) + c.Assert(s.observeWorkerEvent(ctx3, startRev), IsNil) }() c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { bounds := s.BoundSources() @@ -1722,7 +1771,7 @@ func (t *testScheduler) TestWatchLoadTask(c *C) { wg.Add(1) go func() { defer wg.Done() - c.Assert(s.observeLoadTask(ctx1, etcdTestCli, startRev), IsNil) + c.Assert(s.observeLoadTask(ctx1, startRev), IsNil) }() // put task2, source1, worker1 @@ -1813,9 +1862,9 @@ func (t *testScheduler) TestWorkerHasDiffRelayAndBound(c *C) { go ha.KeepAlive(ctx, etcdTestCli, workerName1, keepAlive) // bootstrap - c.Assert(s.recoverSources(etcdTestCli), IsNil) - c.Assert(s.recoverRelayConfigs(etcdTestCli), IsNil) - _, err = s.recoverWorkersBounds(etcdTestCli) + c.Assert(s.recoverSources(), IsNil) + c.Assert(s.recoverRelayConfigs(), IsNil) + _, err = s.recoverWorkersBounds() c.Assert(err, IsNil) // check @@ -1876,9 +1925,9 @@ func (t *testScheduler) TestUpgradeCauseConflictRelayType(c *C) { go ha.KeepAlive(ctx, etcdTestCli, workerName2, keepAlive) // bootstrap - c.Assert(s.recoverSources(etcdTestCli), IsNil) - c.Assert(s.recoverRelayConfigs(etcdTestCli), IsNil) - _, err = s.recoverWorkersBounds(etcdTestCli) + c.Assert(s.recoverSources(), IsNil) + c.Assert(s.recoverRelayConfigs(), IsNil) + _, err = s.recoverWorkersBounds() c.Assert(err, IsNil) // check when the relay config is conflicting with source config, relay config should be deleted diff --git a/dm/dm/master/server.go b/dm/dm/master/server.go index b9de171296b..48691c15c8e 100644 --- a/dm/dm/master/server.go +++ b/dm/dm/master/server.go @@ -132,7 +132,7 @@ func NewServer(cfg *Config) *Server { ap: NewAgentPool(&RateLimitConfig{rate: cfg.RPCRateLimit, burst: cfg.RPCRateBurst}), } server.pessimist = shardddl.NewPessimist(&logger, server.getTaskResources) - server.optimist = shardddl.NewOptimist(&logger) + server.optimist = shardddl.NewOptimist(&logger, server.scheduler.GetDownstreamMetaByTask) server.closed.Store(true) setUseTLS(&cfg.Security) @@ -428,12 +428,23 @@ func (s *Server) StartTask(ctx context.Context, req *pb.StartTaskRequest) (*pb.S } resp := &pb.StartTaskResponse{} - cfg, stCfgs, err := s.generateSubTask(ctx, req.Task) - if err != nil { - resp.Msg = err.Error() + respWithErr := func(err error) (*pb.StartTaskResponse, error) { + resp.Msg += err.Error() // nolint:nilerr return resp, nil } + + cliArgs := config.TaskCliArgs{ + StartTime: req.StartTime, + } + if err := cliArgs.Verify(); err != nil { + return respWithErr(err) + } + + cfg, stCfgs, err := s.generateSubTask(ctx, req.Task, &cliArgs) + if err != nil { + return respWithErr(err) + } msg, err := checker.CheckSyncConfigFunc(ctx, stCfgs, ctlcommon.DefaultErrorCnt, ctlcommon.DefaultWarnCnt) if err != nil { resp.Msg = terror.WithClass(err, terror.ClassDMMaster).Error() @@ -481,29 +492,35 @@ func (s *Server) StartTask(ctx context.Context, req *pb.StartTaskRequest) (*pb.S // use same latch for remove-meta and start-task release, err3 = s.scheduler.AcquireSubtaskLatch(cfg.Name) if err3 != nil { - resp.Msg += terror.ErrSchedulerLatchInUse.Generate("RemoveMeta", cfg.Name).Error() - // nolint:nilerr - return resp, nil + return respWithErr(terror.ErrSchedulerLatchInUse.Generate("RemoveMeta", cfg.Name)) } defer release() latched = true if scm := s.scheduler.GetSubTaskCfgsByTask(cfg.Name); len(scm) > 0 { - resp.Msg += terror.Annotate(terror.ErrSchedulerSubTaskExist.Generate(cfg.Name, sources), - "while remove-meta is true").Error() - return resp, nil + return respWithErr(terror.Annotate(terror.ErrSchedulerSubTaskExist.Generate(cfg.Name, sources), + "while remove-meta is true")) } err = s.removeMetaData(ctx, cfg.Name, cfg.MetaSchema, cfg.TargetDB) if err != nil { - resp.Msg += terror.Annotate(err, "while removing metadata").Error() - return resp, nil + return respWithErr(terror.Annotate(err, "while removing metadata")) + } + } + + if req.StartTime == "" { + err = ha.DeleteAllTaskCliArgs(s.etcdClient, cfg.Name) + if err != nil { + return respWithErr(terror.Annotate(err, "while removing task command line arguments")) + } + } else { + err = ha.PutTaskCliArgs(s.etcdClient, cfg.Name, sources, cliArgs) + if err != nil { + return respWithErr(terror.Annotate(err, "while putting task command line arguments")) } } err = s.scheduler.AddSubTasks(latched, subtaskCfgPointersToInstances(stCfgs...)...) if err != nil { - resp.Msg += err.Error() - // nolint:nilerr - return resp, nil + return respWithErr(err) } if release != nil { @@ -573,6 +590,18 @@ func (s *Server) OperateTask(ctx context.Context, req *pb.OperateTaskRequest) (* resp.Result = true resp.Sources = s.getSourceRespsAfterOperation(ctx, req.Name, sources, []string{}, req) + + if expect == pb.Stage_Stopped { + // delete meta data for optimist + if len(req.Sources) == 0 { + err2 = s.optimist.RemoveMetaDataWithTask(req.Name) + } else { + err2 = s.optimist.RemoveMetaDataWithTaskAndSources(req.Name, sources...) + } + if err2 != nil { + log.L().Error("failed to delete metadata for task", zap.String("task name", req.Name), log.ShortError(err2)) + } + } return resp, nil } @@ -627,8 +656,8 @@ func (s *Server) UpdateTask(ctx context.Context, req *pb.UpdateTaskRequest) (*pb return resp2, err2 } + cfg, stCfgs, err := s.generateSubTask(ctx, req.Task, nil) resp := &pb.UpdateTaskResponse{} - cfg, stCfgs, err := s.generateSubTask(ctx, req.Task) if err != nil { resp.Msg = err.Error() // nolint:nilerr @@ -1190,8 +1219,18 @@ func (s *Server) CheckTask(ctx context.Context, req *pb.CheckTaskRequest) (*pb.C return resp2, err2 } + cliArgs := config.TaskCliArgs{ + StartTime: req.StartTime, + } + if err := cliArgs.Verify(); err != nil { + // nolint:nilerr + return &pb.CheckTaskResponse{ + Result: false, + Msg: err.Error(), + }, nil + } resp := &pb.CheckTaskResponse{} - _, stCfgs, err := s.generateSubTask(ctx, req.Task) + _, stCfgs, err := s.generateSubTask(ctx, req.Task, nil) if err != nil { resp.Msg = err.Error() // nolint:nilerr @@ -1325,7 +1364,12 @@ func (s *Server) OperateSource(ctx context.Context, req *pb.OperateSourceRequest err error ) for _, cfg := range cfgs { - err = s.scheduler.AddSourceCfg(cfg) + // add source with worker when specify a worker name + if req.WorkerName != "" { + err = s.scheduler.AddSourceCfgWithWorker(cfg, req.WorkerName) + } else { + err = s.scheduler.AddSourceCfg(cfg) + } // return first error and try to revert, so user could copy-paste same start command after error if err != nil { resp.Msg = err.Error() @@ -1453,8 +1497,18 @@ func (s *Server) OperateLeader(ctx context.Context, req *pb.OperateLeaderRequest }, nil } -func (s *Server) generateSubTask(ctx context.Context, task string) (*config.TaskConfig, []*config.SubTaskConfig, error) { +func (s *Server) generateSubTask( + ctx context.Context, + task string, + cliArgs *config.TaskCliArgs, +) (*config.TaskConfig, []*config.SubTaskConfig, error) { cfg := config.NewTaskConfig() + // bypass the meta check by set any value. If start-time is specified, DM-worker will not use meta field. + if cliArgs != nil && cliArgs.StartTime != "" { + for _, inst := range cfg.MySQLInstances { + inst.Meta = &config.Meta{BinLogName: cliArgs.StartTime} + } + } err := cfg.Decode(task) if err != nil { return nil, nil, terror.WithClass(err, terror.ClassDMMaster) @@ -1516,7 +1570,7 @@ func (s *Server) removeMetaData(ctx context.Context, taskName, metaSchema string if err != nil { return err } - err = s.optimist.RemoveMetaData(taskName) + err = s.optimist.RemoveMetaDataWithTask(taskName) if err != nil { return err } diff --git a/dm/dm/master/server_test.go b/dm/dm/master/server_test.go index f6e7fe9f370..1438246ef44 100644 --- a/dm/dm/master/server_test.go +++ b/dm/dm/master/server_test.go @@ -857,6 +857,7 @@ func (t *testMaster) TestStartTask(c *check.C) { defer ctrl.Finish() server := testDefaultMasterServer(c) + server.etcdClient = t.etcdTestCli sources, workers := defaultWorkerSource() // s.generateSubTask with error @@ -959,7 +960,7 @@ func (t *testMaster) TestStartTaskWithRemoveMeta(c *check.C) { server.scheduler, _ = t.testMockScheduler(ctx, &wg, c, sources, workers, "", makeWorkerClientsForHandle(ctrl, taskName, sources, workers, req)) server.pessimist = shardddl.NewPessimist(&logger, func(task string) []string { return sources }) - server.optimist = shardddl.NewOptimist(&logger) + server.optimist = shardddl.NewOptimist(&logger, server.scheduler.GetDownstreamMetaByTask) var ( DDLs = []string{"ALTER TABLE bar ADD COLUMN c1 INT"} @@ -1044,7 +1045,7 @@ func (t *testMaster) TestStartTaskWithRemoveMeta(c *check.C) { server.scheduler, _ = t.testMockScheduler(ctx, &wg, c, sources, workers, "", makeWorkerClientsForHandle(ctrl, taskName, sources, workers, req)) server.pessimist = shardddl.NewPessimist(&logger, func(task string) []string { return sources }) - server.optimist = shardddl.NewOptimist(&logger) + server.optimist = shardddl.NewOptimist(&logger, server.scheduler.GetDownstreamMetaByTask) var ( p = parser.New() @@ -1138,6 +1139,7 @@ func (t *testMaster) TestOperateTask(c *check.C) { ctrl := gomock.NewController(c) defer ctrl.Finish() server := testDefaultMasterServer(c) + server.etcdClient = t.etcdTestCli sources, workers := defaultWorkerSource() // test operate-task with invalid task name diff --git a/dm/dm/master/shardddl/optimist.go b/dm/dm/master/shardddl/optimist.go index bab8ef7e86d..58a899b4c05 100644 --- a/dm/dm/master/shardddl/optimist.go +++ b/dm/dm/master/shardddl/optimist.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb-tools/pkg/dbutil" - "github.com/pingcap/tidb-tools/pkg/schemacmp" "go.etcd.io/etcd/clientv3" "go.uber.org/zap" @@ -53,11 +52,11 @@ type Optimist struct { } // NewOptimist creates a new Optimist instance. -func NewOptimist(pLogger *log.Logger) *Optimist { +func NewOptimist(pLogger *log.Logger, getDownstreamMetaFunc func(string) (*config.DBConfig, string)) *Optimist { return &Optimist{ logger: pLogger.WithFields(zap.String("component", "shard DDL optimist")), closed: true, - lk: optimism.NewLockKeeper(), + lk: optimism.NewLockKeeper(getDownstreamMetaFunc), tk: optimism.NewTableKeeper(), } } @@ -162,9 +161,11 @@ func (o *Optimist) ShowLocks(task string, sources []string) []*pb.DDLLock { return ret } -// RemoveMetaData removes meta data for a specified task +// RemoveMetaDataWithTask removes meta data for a specified task // NOTE: this function can only be used when the specified task is not running. -func (o *Optimist) RemoveMetaData(task string) error { +// This function only be used when --remove-meta or stop-task +// NOTE: For stop-task, we still delete drop columns in etcd though user may restart the task again later. +func (o *Optimist) RemoveMetaDataWithTask(task string) error { o.mu.Lock() defer o.mu.Unlock() if o.closed { @@ -185,10 +186,43 @@ func (o *Optimist) RemoveMetaData(task string) error { o.lk.RemoveLock(op.ID) } + o.lk.RemoveDownstreamMeta(task) o.tk.RemoveTableByTask(task) // clear meta data in etcd - _, err = optimism.DeleteInfosOperationsTablesSchemasByTask(o.cli, task, lockIDSet) + _, err = optimism.DeleteInfosOperationsTablesByTask(o.cli, task, lockIDSet) + return err +} + +// RemoveMetaDataWithTaskAndSources removes meta data for a specified task and sources +// NOTE: this function can only be used when the specified task for source is not running. +func (o *Optimist) RemoveMetaDataWithTaskAndSources(task string, sources ...string) error { + o.mu.Lock() + defer o.mu.Unlock() + if o.closed { + return terror.ErrMasterOptimistNotStarted.Generate() + } + + dropColumns := make(map[string][]string) + + // gets all locks for this task + locks := o.lk.FindLocksByTask(task) + for _, lock := range locks { + // remove table by sources for related lock + cols := lock.TryRemoveTableBySources(sources) + dropColumns[lock.ID] = cols + o.logger.Debug("the tables removed from the lock", zap.String("task", task), zap.Strings("sources", sources)) + if !lock.HasTables() { + o.lk.RemoveLock(lock.ID) + } + } + + o.lk.RemoveDownstreamMeta(task) + // remove source table in table keeper + o.tk.RemoveTableByTaskAndSources(task, sources) + o.logger.Debug("the tables removed from the table keeper", zap.String("task", task), zap.Strings("source", sources)) + // clear meta data in etcd + _, err := optimism.DeleteInfosOperationsTablesByTaskAndSource(o.cli, task, sources, dropColumns) return err } @@ -250,26 +284,22 @@ func (o *Optimist) rebuildLocks() (revSource, revInfo, revOperation int64, err e } o.logger.Info("get history shard DDL lock operation", zap.Int64("revision", revOperation)) - initSchemas, revInitSchemas, err := optimism.GetAllInitSchemas(o.cli) - if err != nil { - return 0, 0, 0, err - } - o.logger.Info("get all init schemas", zap.Int64("revision", revInitSchemas)) - colm, _, err := optimism.GetAllDroppedColumns(o.cli) if err != nil { // only log the error, and don't return it to forbid the startup of the DM-master leader. // then these unexpected columns can be handled by the user. o.logger.Error("fail to recover colms", log.ShortError(err)) } + o.lk.SetDropColumns(colm) // recover the shard DDL lock based on history shard DDL info & lock operation. - err = o.recoverLocks(ifm, opm, colm, initSchemas) + err = o.recoverLocks(ifm, opm) if err != nil { // only log the error, and don't return it to forbid the startup of the DM-master leader. // then these unexpected locks can be handled by the user. o.logger.Error("fail to recover locks", log.ShortError(err)) } + o.lk.SetDropColumns(nil) return revSource, revInfo, revOperation, nil } @@ -295,96 +325,11 @@ func sortInfos(ifm map[string]map[string]map[string]map[string]optimism.Info) [] return infos } -// buildLockJoinedAndTTS build joined table and target table slice for lock by history infos. -func (o *Optimist) buildLockJoinedAndTTS( - ifm map[string]map[string]map[string]map[string]optimism.Info, - initSchemas map[string]map[string]map[string]optimism.InitSchema) ( - map[string]schemacmp.Table, map[string][]optimism.TargetTable, - map[string]map[string]map[string]map[string]schemacmp.Table) { - type infoKey struct { - lockID string - source string - upSchema string - upTable string - } - infos := make(map[infoKey]optimism.Info) - lockTTS := make(map[string][]optimism.TargetTable) - for _, taskInfos := range ifm { - for _, sourceInfos := range taskInfos { - for _, schemaInfos := range sourceInfos { - for _, info := range schemaInfos { - lockID := utils.GenDDLLockID(info.Task, info.DownSchema, info.DownTable) - if _, ok := lockTTS[lockID]; !ok { - lockTTS[lockID] = o.tk.FindTables(info.Task, info.DownSchema, info.DownTable) - } - infos[infoKey{lockID, info.Source, info.UpSchema, info.UpTable}] = info - } - } - } - } - - lockJoined := make(map[string]schemacmp.Table) - missTable := make(map[string]map[string]map[string]map[string]schemacmp.Table) - for lockID, tts := range lockTTS { - for _, tt := range tts { - for upSchema, tables := range tt.UpTables { - for upTable := range tables { - var table schemacmp.Table - if info, ok := infos[infoKey{lockID, tt.Source, upSchema, upTable}]; ok && info.TableInfoBefore != nil { - table = schemacmp.Encode(info.TableInfoBefore) - } else if initSchema, ok := initSchemas[tt.Task][tt.DownSchema][tt.DownTable]; ok { - // If there is no optimism.Info for a upstream table, it indicates the table structure - // hasn't been changed since last removeLock. So the init schema should be its table info. - table = schemacmp.Encode(initSchema.TableInfo) - if _, ok := missTable[lockID]; !ok { - missTable[lockID] = make(map[string]map[string]map[string]schemacmp.Table) - } - if _, ok := missTable[lockID][tt.Source]; !ok { - missTable[lockID][tt.Source] = make(map[string]map[string]schemacmp.Table) - } - if _, ok := missTable[lockID][tt.Source][upSchema]; !ok { - missTable[lockID][tt.Source][upSchema] = make(map[string]schemacmp.Table) - } - missTable[lockID][tt.Source][upSchema][upTable] = table - } else { - o.logger.Error( - "can not find table info for upstream table", - zap.String("source", tt.Source), - zap.String("up-schema", upSchema), - zap.String("up-table", upTable), - ) - continue - } - if joined, ok := lockJoined[lockID]; !ok { - lockJoined[lockID] = table - } else { - newJoined, err := joined.Join(table) - // ignore error, will report it in TrySync later - if err != nil { - o.logger.Error(fmt.Sprintf("fail to join table info %s with %s, lockID: %s in recover lock", joined, newJoined, lockID), log.ShortError(err)) - } else { - lockJoined[lockID] = newJoined - } - } - } - } - } - } - return lockJoined, lockTTS, missTable -} - // recoverLocks recovers shard DDL locks based on shard DDL info and shard DDL lock operation. func (o *Optimist) recoverLocks( ifm map[string]map[string]map[string]map[string]optimism.Info, opm map[string]map[string]map[string]map[string]optimism.Operation, - colm map[string]map[string]map[string]map[string]map[string]optimism.DropColumnStage, - initSchemas map[string]map[string]map[string]optimism.InitSchema) error { - // construct joined table based on the shard DDL info. - o.logger.Info("build lock joined and tts") - lockJoined, lockTTS, missTable := o.buildLockJoinedAndTTS(ifm, initSchemas) - // build lock and restore table info - o.logger.Info("rebuild locks and tables") - o.lk.RebuildLocksAndTables(o.cli, ifm, colm, lockJoined, lockTTS, missTable) +) error { // sort infos by revision infos := sortInfos(ifm) var firstErr error @@ -395,12 +340,18 @@ func (o *Optimist) recoverLocks( } for _, info := range infos { + if info.IsDeleted { + // TODO: handle drop table + continue + } + if !o.tk.SourceTableExist(info.Task, info.Source, info.UpSchema, info.UpTable, info.DownSchema, info.DownTable) { + continue + } // never mark the lock operation from `done` to `not-done` when recovering. err := o.handleInfo(info, true) if err != nil { o.logger.Error("fail to handle info while recovering locks", zap.Error(err)) setFirstErr(err) - continue } } @@ -419,7 +370,6 @@ func (o *Optimist) recoverLocks( err := lock.DeleteColumnsByOp(op) if err != nil { o.logger.Error("fail to update lock columns", zap.Error(err)) - continue } } } @@ -595,43 +545,45 @@ func (o *Optimist) handleOperationPut(ctx context.Context, opCh <-chan optimism. // avoid new ddl added while previous ddl resolved and remove lock // change lock granularity if needed o.mu.Lock() - lock := o.lk.FindLock(op.ID) - if lock == nil { - o.logger.Warn("no lock for the shard DDL lock operation exist", zap.Stringer("operation", op)) - o.mu.Unlock() - continue - } - - err := lock.DeleteColumnsByOp(op) - if err != nil { - o.logger.Error("fail to update lock columns", zap.Error(err)) - } - // in optimistic mode, we always try to mark a table as done after received the `done` status of the DDLs operation. - // NOTE: even all tables have done their previous DDLs operations, the lock may still not resolved, - // because these tables may have different schemas. - done := lock.TryMarkDone(op.Source, op.UpSchema, op.UpTable) - o.logger.Info("mark operation for a table as done", zap.Bool("done", done), zap.Stringer("operation", op)) - if !lock.IsResolved() { - o.logger.Info("the lock is still not resolved", zap.Stringer("operation", op)) - o.mu.Unlock() - continue - } - - // the lock has done, remove the lock. - o.logger.Info("the lock for the shard DDL lock operation has been resolved", zap.Stringer("operation", op)) - deleted, err := o.removeLock(lock) - if err != nil { - o.logger.Error("fail to delete the shard DDL infos and lock operations", zap.String("lock", lock.ID), log.ShortError(err)) - metrics.ReportDDLError(op.Task, metrics.OpErrRemoveLock) - } - if deleted { - o.logger.Info("the shard DDL infos and lock operations have been cleared", zap.Stringer("operation", op)) - } + o.handleOperation(op) o.mu.Unlock() } } } +func (o *Optimist) handleOperation(op optimism.Operation) { + lock := o.lk.FindLock(op.ID) + if lock == nil { + o.logger.Warn("no lock for the shard DDL lock operation exist", zap.Stringer("operation", op)) + return + } + + err := lock.DeleteColumnsByOp(op) + if err != nil { + o.logger.Error("fail to update lock columns", zap.Error(err)) + } + // in optimistic mode, we always try to mark a table as done after received the `done` status of the DDLs operation. + // NOTE: even all tables have done their previous DDLs operations, the lock may still not resolved, + // because these tables may have different schemas. + done := lock.TryMarkDone(op.Source, op.UpSchema, op.UpTable) + o.logger.Info("mark operation for a table as done", zap.Bool("done", done), zap.Stringer("operation", op)) + if !lock.IsResolved() { + o.logger.Info("the lock is still not resolved", zap.Stringer("operation", op)) + return + } + + // the lock has done, remove the lock. + o.logger.Info("the lock for the shard DDL lock operation has been resolved", zap.Stringer("operation", op)) + deleted, err := o.removeLock(lock) + if err != nil { + o.logger.Error("fail to delete the shard DDL infos and lock operations", zap.String("lock", lock.ID), log.ShortError(err)) + metrics.ReportDDLError(op.Task, metrics.OpErrRemoveLock) + } + if deleted { + o.logger.Info("the shard DDL infos and lock operations have been cleared", zap.Stringer("operation", op)) + } +} + // handleLock handles a single shard DDL lock. func (o *Optimist) handleLock(info optimism.Info, tts []optimism.TargetTable, skipDone bool) error { cfStage := optimism.ConflictNone @@ -649,18 +601,6 @@ func (o *Optimist) handleLock(info optimism.Info, tts []optimism.TargetTable, sk default: o.logger.Info("the shard DDL lock returned some DDLs", zap.String("lock", lockID), zap.Strings("ddls", newDDLs), zap.Strings("cols", cols), zap.String("info", info.ShortString()), zap.Bool("is deleted", info.IsDeleted)) - - // try to record the init schema before applied the DDL to the downstream. - initSchema := optimism.NewInitSchema(info.Task, info.DownSchema, info.DownTable, info.TableInfoBefore) - rev, putted, err2 := optimism.PutInitSchemaIfNotExist(o.cli, initSchema) - switch { - case err2 != nil: - return err2 - case putted: - o.logger.Info("recorded the initial schema", zap.String("info", info.ShortString())) - default: - o.logger.Debug("skip to record the initial schema", zap.String("info", info.ShortString()), zap.Int64("revision", rev)) - } } lock := o.lk.FindLock(lockID) @@ -761,8 +701,7 @@ func (o *Optimist) deleteInfosOps(lock *optimism.Lock) (bool, error) { } } // NOTE: we rely on only `task`, `downSchema`, and `downTable` used for deletion. - initSchema := optimism.NewInitSchema(lock.Task, lock.DownSchema, lock.DownTable, nil) - rev, deleted, err := optimism.DeleteInfosOperationsSchemaColumn(o.cli, infos, ops, initSchema) + rev, deleted, err := optimism.DeleteInfosOperationsColumns(o.cli, infos, ops, lock.ID) if err != nil { return deleted, err } diff --git a/dm/dm/master/shardddl/optimist_test.go b/dm/dm/master/shardddl/optimist_test.go index d8226ee996c..b3e6c84e13e 100644 --- a/dm/dm/master/shardddl/optimist_test.go +++ b/dm/dm/master/shardddl/optimist_test.go @@ -20,7 +20,6 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb-tools/pkg/dbutil" - "github.com/pingcap/tidb-tools/pkg/schemacmp" tiddl "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" @@ -43,7 +42,7 @@ var _ = SerialSuites(&testOptimist{}) // clear keys in etcd test cluster. func clearOptimistTestSourceInfoOperation(c *C) { - c.Assert(optimism.ClearTestInfoOperationSchema(etcdTestCli), IsNil) + c.Assert(optimism.ClearTestInfoOperationColumn(etcdTestCli), IsNil) } func createTableInfo(c *C, p *parser.Parser, se sessionctx.Context, tableID int64, sql string) *model.TableInfo { @@ -104,7 +103,7 @@ func (t *testOptimist) TestOptimistSourceTables(c *C) { var ( logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) task = "task" source1 = "mysql-replica-1" source2 = "mysql-replica-2" @@ -164,7 +163,7 @@ func (t *testOptimist) TestOptimistSourceTables(c *C) { o.Close() // CASE 4: create (not re-start) a new optimist with previous source tables. - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) c.Assert(o.Start(ctx, etcdTestCli), IsNil) tts = o.tk.FindTables(task, downSchema, downTable) c.Assert(tts, HasLen, 2) @@ -197,14 +196,14 @@ func (t *testOptimist) TestOptimist(c *C) { func (t *testOptimist) testOptimist(c *C, cli *clientv3.Client, restart int) { defer func() { - c.Assert(optimism.ClearTestInfoOperationSchema(cli), IsNil) + c.Assert(optimism.ClearTestInfoOperationColumn(cli), IsNil) }() var ( backOff = 30 waitTime = 100 * time.Millisecond logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) rebuildOptimist = func(ctx context.Context) { switch restart { @@ -213,7 +212,7 @@ func (t *testOptimist) testOptimist(c *C, cli *clientv3.Client, restart int) { c.Assert(o.Start(ctx, cli), IsNil) case restartNewInstance: o.Close() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) c.Assert(o.Start(ctx, cli), IsNil) } } @@ -615,7 +614,7 @@ func (t *testOptimist) TestOptimistLockConflict(c *C) { var ( watchTimeout = 5 * time.Second logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) task = "task-test-optimist" source1 = "mysql-replica-1" downSchema = "foo" @@ -724,7 +723,7 @@ func (t *testOptimist) TestOptimistLockMultipleTarget(c *C) { waitTime = 100 * time.Millisecond watchTimeout = 5 * time.Second logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) task = "test-optimist-lock-multiple-target" source = "mysql-replica-1" upSchema = "foo" @@ -919,7 +918,7 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { waitTime = 100 * time.Millisecond watchTimeout = 5 * time.Second logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) task = "test-optimist-init-schema" source = "mysql-replica-1" upSchema = "foo" @@ -954,11 +953,6 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { c.Assert(o.Start(ctx, etcdTestCli), IsNil) c.Assert(o.Locks(), HasLen, 0) - // no init schema exist now. - is, _, err := optimism.GetInitSchema(etcdTestCli, task, downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(is.IsEmpty(), IsTrue) - // PUT i11, will creat a lock. _, err = optimism.PutInfo(etcdTestCli, i11) c.Assert(err, IsNil) @@ -967,11 +961,6 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { }), IsTrue) time.Sleep(waitTime) // sleep one more time to wait for update of init schema. - // the init schema exist now. - is, _, err = optimism.GetInitSchema(etcdTestCli, task, downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(is.TableInfo, DeepEquals, ti0) // the init schema. - // PUT i12, the lock will be synced. rev1, err := optimism.PutInfo(etcdTestCli, i12) c.Assert(err, IsNil) @@ -1011,11 +1000,6 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { return len(o.Locks()) == 0 }), IsTrue) - // the init schema should also be deleted. - is, _, err = optimism.GetInitSchema(etcdTestCli, task, downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(is.IsEmpty(), IsTrue) - // PUT i21 to create the lock again. _, err = optimism.PutInfo(etcdTestCli, i21) c.Assert(err, IsNil) @@ -1023,16 +1007,11 @@ func (t *testOptimist) TestOptimistInitSchema(c *C) { return len(o.Locks()) == 1 }), IsTrue) time.Sleep(waitTime) // sleep one more time to wait for update of init schema. - - // the init schema exist now. - is, _, err = optimism.GetInitSchema(etcdTestCli, task, downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(is.TableInfo, DeepEquals, ti1) // the init schema is ti1 now. } func (t *testOptimist) testSortInfos(c *C, cli *clientv3.Client) { defer func() { - c.Assert(optimism.ClearTestInfoOperationSchema(cli), IsNil) + c.Assert(optimism.ClearTestInfoOperationColumn(cli), IsNil) }() var ( @@ -1104,7 +1083,7 @@ func (t *testOptimist) TestBuildLockJoinedAndTable(c *C) { var ( logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) task = "task" source1 = "mysql-replica-1" source2 = "mysql-replica-2" @@ -1146,19 +1125,6 @@ func (t *testOptimist) TestBuildLockJoinedAndTable(c *C) { stm, _, err := optimism.GetAllSourceTables(etcdTestCli) c.Assert(err, IsNil) o.tk.Init(stm) - - ifm, _, err := optimism.GetAllInfo(etcdTestCli) - c.Assert(err, IsNil) - - lockJoined, lockTTS, missTable := o.buildLockJoinedAndTTS(ifm, nil) - c.Assert(len(lockJoined), Equals, 1) - c.Assert(len(lockTTS), Equals, 1) - c.Assert(len(missTable), Equals, 0) - joined, ok := lockJoined[utils.GenDDLLockID(task, downSchema, downTable)] - c.Assert(ok, IsTrue) - cmp, err := joined.Compare(schemacmp.Encode(ti2)) - c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) } func (t *testOptimist) TestBuildLockWithInitSchema(c *C) { @@ -1166,7 +1132,7 @@ func (t *testOptimist) TestBuildLockWithInitSchema(c *C) { var ( logger = log.L() - o = NewOptimist(&logger) + o = NewOptimist(&logger, getDownstreamMeta) task = "task" source1 = "mysql-replica-1" source2 = "mysql-replica-2" @@ -1182,11 +1148,10 @@ func (t *testOptimist) TestBuildLockWithInitSchema(c *C) { ti1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (a INT PRIMARY KEY, b INT)`) ti2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (a INT PRIMARY KEY)`) - ddlDropB = "ALTER TABLE bar DROP COLUMN b" - ddlDropC = "ALTER TABLE bar DROP COLUMN c" - infoDropB = optimism.NewInfo(task, source1, "foo", "bar-1", downSchema, downTable, []string{ddlDropC}, ti0, []*model.TableInfo{ti1}) - infoDropC = optimism.NewInfo(task, source1, "foo", "bar-1", downSchema, downTable, []string{ddlDropB}, ti1, []*model.TableInfo{ti2}) - initSchema = optimism.NewInitSchema(task, downSchema, downTable, ti0) + ddlDropB = "ALTER TABLE bar DROP COLUMN b" + ddlDropC = "ALTER TABLE bar DROP COLUMN c" + infoDropB = optimism.NewInfo(task, source1, "foo", "bar-1", downSchema, downTable, []string{ddlDropC}, ti0, []*model.TableInfo{ti1}) + infoDropC = optimism.NewInfo(task, source1, "foo", "bar-1", downSchema, downTable, []string{ddlDropB}, ti1, []*model.TableInfo{ti2}) ) ctx, cancel := context.WithCancel(context.Background()) @@ -1209,21 +1174,8 @@ func (t *testOptimist) TestBuildLockWithInitSchema(c *C) { stm, _, err := optimism.GetAllSourceTables(etcdTestCli) c.Assert(err, IsNil) o.tk.Init(stm) +} - ifm, _, err := optimism.GetAllInfo(etcdTestCli) - c.Assert(err, IsNil) - - initSchemas := map[string]map[string]map[string]optimism.InitSchema{task: {downSchema: {downTable: initSchema}}} - lockJoined, lockTTS, missTable := o.buildLockJoinedAndTTS(ifm, initSchemas) - c.Assert(len(lockJoined), Equals, 1) - c.Assert(len(lockTTS), Equals, 1) - c.Assert(len(missTable), Equals, 1) - cmp, err := missTable[utils.GenDDLLockID(task, downSchema, downTable)][source2]["foo"]["bar-1"].Compare(schemacmp.Encode(initSchema.TableInfo)) - c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) - joined, ok := lockJoined[utils.GenDDLLockID(task, downSchema, downTable)] - c.Assert(ok, IsTrue) - cmp, err = joined.Compare(schemacmp.Encode(ti0)) - c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) +func getDownstreamMeta(string) (*config.DBConfig, string) { + return nil, "" } diff --git a/dm/dm/pb/dmmaster.pb.go b/dm/dm/pb/dmmaster.pb.go index 3ace56ccdef..b70beb74973 100644 --- a/dm/dm/pb/dmmaster.pb.go +++ b/dm/dm/pb/dmmaster.pb.go @@ -158,6 +158,7 @@ type StartTaskRequest struct { Task string `protobuf:"bytes,1,opt,name=task,proto3" json:"task,omitempty"` Sources []string `protobuf:"bytes,2,rep,name=sources,proto3" json:"sources,omitempty"` RemoveMeta bool `protobuf:"varint,3,opt,name=removeMeta,proto3" json:"removeMeta,omitempty"` + StartTime string `protobuf:"bytes,4,opt,name=startTime,proto3" json:"startTime,omitempty"` } func (m *StartTaskRequest) Reset() { *m = StartTaskRequest{} } @@ -214,6 +215,13 @@ func (m *StartTaskRequest) GetRemoveMeta() bool { return false } +func (m *StartTaskRequest) GetStartTime() string { + if m != nil { + return m.StartTime + } + return "" +} + type StartTaskResponse struct { Result bool `protobuf:"varint,1,opt,name=result,proto3" json:"result,omitempty"` Msg string `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` @@ -1229,9 +1237,10 @@ func (m *PurgeWorkerRelayResponse) GetSources() []*CommonWorkerResponse { } type CheckTaskRequest struct { - Task string `protobuf:"bytes,1,opt,name=task,proto3" json:"task,omitempty"` - ErrCnt int64 `protobuf:"varint,2,opt,name=errCnt,proto3" json:"errCnt,omitempty"` - WarnCnt int64 `protobuf:"varint,3,opt,name=warnCnt,proto3" json:"warnCnt,omitempty"` + Task string `protobuf:"bytes,1,opt,name=task,proto3" json:"task,omitempty"` + ErrCnt int64 `protobuf:"varint,2,opt,name=errCnt,proto3" json:"errCnt,omitempty"` + WarnCnt int64 `protobuf:"varint,3,opt,name=warnCnt,proto3" json:"warnCnt,omitempty"` + StartTime string `protobuf:"bytes,4,opt,name=startTime,proto3" json:"startTime,omitempty"` } func (m *CheckTaskRequest) Reset() { *m = CheckTaskRequest{} } @@ -1288,6 +1297,13 @@ func (m *CheckTaskRequest) GetWarnCnt() int64 { return 0 } +func (m *CheckTaskRequest) GetStartTime() string { + if m != nil { + return m.StartTime + } + return "" +} + type CheckTaskResponse struct { Result bool `protobuf:"varint,1,opt,name=result,proto3" json:"result,omitempty"` Msg string `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` @@ -1341,9 +1357,10 @@ func (m *CheckTaskResponse) GetMsg() string { } type OperateSourceRequest struct { - Op SourceOp `protobuf:"varint,1,opt,name=op,proto3,enum=pb.SourceOp" json:"op,omitempty"` - Config []string `protobuf:"bytes,2,rep,name=config,proto3" json:"config,omitempty"` - SourceID []string `protobuf:"bytes,3,rep,name=sourceID,proto3" json:"sourceID,omitempty"` + Op SourceOp `protobuf:"varint,1,opt,name=op,proto3,enum=pb.SourceOp" json:"op,omitempty"` + Config []string `protobuf:"bytes,2,rep,name=config,proto3" json:"config,omitempty"` + SourceID []string `protobuf:"bytes,3,rep,name=sourceID,proto3" json:"sourceID,omitempty"` + WorkerName string `protobuf:"bytes,4,opt,name=workerName,proto3" json:"workerName,omitempty"` } func (m *OperateSourceRequest) Reset() { *m = OperateSourceRequest{} } @@ -1400,6 +1417,13 @@ func (m *OperateSourceRequest) GetSourceID() []string { return nil } +func (m *OperateSourceRequest) GetWorkerName() string { + if m != nil { + return m.WorkerName + } + return "" +} + type OperateSourceResponse struct { Result bool `protobuf:"varint,1,opt,name=result,proto3" json:"result,omitempty"` Msg string `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` @@ -3190,137 +3214,139 @@ func init() { func init() { proto.RegisterFile("dmmaster.proto", fileDescriptor_f9bef11f2a341f03) } var fileDescriptor_f9bef11f2a341f03 = []byte{ - // 2074 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x59, 0x5f, 0x6f, 0xe3, 0xc6, - 0x11, 0x17, 0x25, 0x5b, 0x96, 0x47, 0xb6, 0x22, 0xaf, 0x65, 0x99, 0xc7, 0xf3, 0xe9, 0x1c, 0x36, - 0x09, 0x0c, 0xa3, 0x38, 0xe3, 0xdc, 0x3e, 0x05, 0x48, 0x81, 0x9c, 0x74, 0xb9, 0x18, 0xf5, 0xd5, - 0x29, 0xed, 0x4b, 0x1b, 0x14, 0x28, 0x42, 0x49, 0x2b, 0x59, 0x30, 0x45, 0xf2, 0x48, 0xca, 0xae, - 0x71, 0xc8, 0x4b, 0x3f, 0x40, 0xff, 0xa0, 0x40, 0xf3, 0xd8, 0x87, 0x7e, 0x93, 0x3e, 0xf5, 0x31, - 0x40, 0x5f, 0xfa, 0x58, 0xdc, 0xf5, 0x83, 0x14, 0x3b, 0xb3, 0x24, 0x97, 0x14, 0xe5, 0x56, 0x01, - 0xea, 0x37, 0xce, 0xcc, 0x6a, 0xe6, 0x37, 0x7f, 0x76, 0x76, 0x76, 0x05, 0x8d, 0xe1, 0x74, 0x6a, - 0x87, 0x11, 0x0f, 0x9e, 0xf8, 0x81, 0x17, 0x79, 0xac, 0xec, 0xf7, 0x8d, 0xc6, 0x70, 0x7a, 0xe3, - 0x05, 0x57, 0x31, 0xcf, 0xd8, 0x1b, 0x7b, 0xde, 0xd8, 0xe1, 0x47, 0xb6, 0x3f, 0x39, 0xb2, 0x5d, - 0xd7, 0x8b, 0xec, 0x68, 0xe2, 0xb9, 0x21, 0x49, 0xcd, 0xaf, 0xa1, 0x79, 0x1e, 0xd9, 0x41, 0x74, - 0x61, 0x87, 0x57, 0x16, 0x7f, 0x3d, 0xe3, 0x61, 0xc4, 0x18, 0xac, 0x44, 0x76, 0x78, 0xa5, 0x6b, - 0xfb, 0xda, 0xc1, 0xba, 0x85, 0xdf, 0x4c, 0x87, 0xb5, 0xd0, 0x9b, 0x05, 0x03, 0x1e, 0xea, 0xe5, - 0xfd, 0xca, 0xc1, 0xba, 0x15, 0x93, 0xac, 0x03, 0x10, 0xf0, 0xa9, 0x77, 0xcd, 0x5f, 0xf2, 0xc8, - 0xd6, 0x2b, 0xfb, 0xda, 0x41, 0xcd, 0x52, 0x38, 0xe6, 0x6b, 0xd8, 0x52, 0x2c, 0x84, 0xbe, 0xe7, - 0x86, 0x9c, 0xb5, 0xa1, 0x1a, 0xf0, 0x70, 0xe6, 0x44, 0x68, 0xa4, 0x66, 0x49, 0x8a, 0x35, 0xa1, - 0x32, 0x0d, 0xc7, 0x7a, 0x19, 0x2d, 0x8b, 0x4f, 0x76, 0x9c, 0x1a, 0xae, 0xec, 0x57, 0x0e, 0xea, - 0xc7, 0xfa, 0x13, 0xbf, 0xff, 0xa4, 0xeb, 0x4d, 0xa7, 0x9e, 0xfb, 0x0b, 0xf4, 0x33, 0x56, 0x9a, - 0x40, 0x32, 0x7f, 0x0d, 0xec, 0xcc, 0xe7, 0x81, 0x1d, 0x71, 0xd5, 0x2d, 0x03, 0xca, 0x9e, 0x8f, - 0xf6, 0x1a, 0xc7, 0x20, 0x94, 0x08, 0xe1, 0x99, 0x6f, 0x95, 0x3d, 0x5f, 0xb8, 0xec, 0xda, 0x53, - 0x2e, 0x0d, 0xe3, 0xb7, 0xea, 0x72, 0x25, 0xe3, 0xb2, 0xf9, 0x7b, 0x0d, 0xb6, 0x33, 0x06, 0xa4, - 0x57, 0x77, 0x59, 0x48, 0x3d, 0x2e, 0x17, 0x79, 0x5c, 0x29, 0xf4, 0x78, 0xe5, 0x7f, 0xf5, 0xf8, - 0x53, 0xd8, 0x7a, 0xe5, 0x0f, 0x73, 0x0e, 0x2f, 0x95, 0x47, 0x33, 0x00, 0xa6, 0xaa, 0xb8, 0x97, - 0x44, 0x7d, 0x06, 0xed, 0x9f, 0xcf, 0x78, 0x70, 0x7b, 0x1e, 0xd9, 0xd1, 0x2c, 0x3c, 0x9d, 0x84, - 0x91, 0x82, 0x1d, 0x13, 0xa2, 0x15, 0x27, 0x24, 0x87, 0xfd, 0x1a, 0x76, 0xe7, 0xf4, 0x2c, 0xed, - 0xc0, 0xd3, 0xbc, 0x03, 0xbb, 0xc2, 0x01, 0x45, 0xef, 0x3c, 0xfe, 0x2e, 0x6c, 0x9f, 0x5f, 0x7a, - 0x37, 0xbd, 0xde, 0xe9, 0xa9, 0x37, 0xb8, 0x0a, 0xbf, 0x5f, 0xe0, 0xff, 0xa2, 0xc1, 0x9a, 0xd4, - 0xc0, 0x1a, 0x50, 0x3e, 0xe9, 0xc9, 0xdf, 0x95, 0x4f, 0x7a, 0x89, 0xa6, 0xb2, 0xa2, 0x89, 0xc1, - 0xca, 0xd4, 0x1b, 0x72, 0x59, 0x32, 0xf8, 0xcd, 0x5a, 0xb0, 0xea, 0xdd, 0xb8, 0x3c, 0xd0, 0x57, - 0x90, 0x49, 0x84, 0x58, 0xd9, 0xeb, 0x9d, 0x86, 0xfa, 0x2a, 0x1a, 0xc4, 0x6f, 0x11, 0x8f, 0xf0, - 0xd6, 0x1d, 0xf0, 0xa1, 0x5e, 0x45, 0xae, 0xa4, 0x98, 0x01, 0xb5, 0x99, 0x2b, 0x25, 0x6b, 0x28, - 0x49, 0x68, 0x73, 0x00, 0xad, 0xac, 0x9b, 0x4b, 0xc7, 0xf6, 0x7d, 0x58, 0x75, 0xc4, 0x4f, 0x65, - 0x64, 0xeb, 0x22, 0xb2, 0x52, 0x9d, 0x45, 0x12, 0xd3, 0x81, 0xd6, 0x2b, 0x57, 0x7c, 0xc6, 0x7c, - 0x19, 0xcc, 0x7c, 0x48, 0x4c, 0xd8, 0x08, 0xb8, 0xef, 0xd8, 0x03, 0x7e, 0x86, 0x1e, 0x93, 0x95, - 0x0c, 0x8f, 0xed, 0x43, 0x7d, 0xe4, 0x05, 0x03, 0x6e, 0x61, 0x1b, 0x92, 0x4d, 0x49, 0x65, 0x99, - 0x9f, 0xc2, 0x4e, 0xce, 0xda, 0xb2, 0x3e, 0x99, 0x16, 0x3c, 0x90, 0x4d, 0x20, 0x2e, 0x6f, 0xc7, - 0xbe, 0x8d, 0x51, 0x3f, 0x54, 0x5a, 0x01, 0x7a, 0x8b, 0x52, 0xd9, 0x0b, 0x16, 0xd7, 0xc2, 0xb7, - 0x1a, 0x18, 0x45, 0x4a, 0x25, 0xb8, 0x3b, 0xb5, 0xfe, 0x7f, 0x3b, 0xcc, 0xb7, 0x1a, 0xec, 0x7e, - 0x31, 0x0b, 0xc6, 0x45, 0xce, 0x2a, 0xfe, 0x68, 0xd9, 0xc3, 0xc1, 0x80, 0xda, 0xc4, 0xb5, 0x07, - 0xd1, 0xe4, 0x9a, 0x4b, 0x54, 0x09, 0x8d, 0xb5, 0x3d, 0x99, 0x52, 0x76, 0x2a, 0x16, 0x7e, 0x8b, - 0xf5, 0xa3, 0x89, 0xc3, 0x71, 0xeb, 0x53, 0x29, 0x27, 0x34, 0x56, 0xee, 0xac, 0xdf, 0x9b, 0x04, - 0xfa, 0x2a, 0x4a, 0x24, 0x65, 0xfe, 0x06, 0xf4, 0x79, 0x60, 0xf7, 0xd2, 0xbe, 0x7e, 0x09, 0xcd, - 0xee, 0x25, 0x1f, 0x5c, 0xfd, 0xb7, 0xa6, 0xdb, 0x86, 0x2a, 0x0f, 0x82, 0xae, 0x4b, 0x99, 0xa9, - 0x58, 0x92, 0x12, 0x71, 0xbb, 0xb1, 0x03, 0x57, 0x08, 0x28, 0x08, 0x31, 0x69, 0x7e, 0x02, 0x5b, - 0x8a, 0xe6, 0xa5, 0x4b, 0xf3, 0x12, 0x5a, 0xb2, 0x8a, 0xce, 0x11, 0x6a, 0x0c, 0x6e, 0x4f, 0xa9, - 0x9f, 0x0d, 0xe1, 0x1f, 0x89, 0xd3, 0x02, 0x1a, 0x78, 0xee, 0x68, 0x32, 0x96, 0x55, 0x29, 0x29, - 0x91, 0x14, 0xf2, 0xf8, 0xa4, 0x27, 0x4f, 0xc2, 0x84, 0x36, 0x67, 0xb0, 0x93, 0xb3, 0x74, 0x2f, - 0x91, 0x7f, 0x0e, 0x3b, 0x16, 0x1f, 0x4f, 0xc4, 0xe8, 0x13, 0x2f, 0xb9, 0xf3, 0xdc, 0xb0, 0x87, - 0xc3, 0x80, 0x87, 0xa1, 0x34, 0x1b, 0x93, 0xe6, 0x33, 0x68, 0xe7, 0xd5, 0x2c, 0x1d, 0xeb, 0x9f, - 0x40, 0xeb, 0x6c, 0x34, 0x72, 0x26, 0x2e, 0x7f, 0xc9, 0xa7, 0xfd, 0x0c, 0x92, 0xe8, 0xd6, 0x4f, - 0x90, 0x88, 0xef, 0xa2, 0x31, 0x43, 0x74, 0xa2, 0xdc, 0xef, 0x97, 0x86, 0xf0, 0xe3, 0x24, 0xdd, - 0xa7, 0xdc, 0x1e, 0xa6, 0x10, 0xe6, 0xd2, 0x4d, 0x62, 0x4a, 0x37, 0x1a, 0xce, 0xfe, 0x6a, 0x69, - 0xc3, 0xbf, 0xd3, 0x00, 0x5e, 0xe2, 0x00, 0x7a, 0xe2, 0x8e, 0xbc, 0xc2, 0xe0, 0x1b, 0x50, 0x9b, - 0xa2, 0x5f, 0x27, 0x3d, 0xfc, 0xe5, 0x8a, 0x95, 0xd0, 0xe2, 0xd4, 0xb2, 0x9d, 0x49, 0xd2, 0xa0, - 0x89, 0x10, 0xbf, 0xf0, 0x39, 0x0f, 0x5e, 0x59, 0xa7, 0xd4, 0x9e, 0xd6, 0xad, 0x84, 0x16, 0xc3, - 0xe6, 0xc0, 0x99, 0x70, 0x37, 0x42, 0x29, 0x9d, 0x6b, 0x0a, 0xc7, 0xec, 0x03, 0x50, 0x22, 0x17, - 0xe2, 0x61, 0xb0, 0x22, 0xb2, 0x1f, 0xa7, 0x40, 0x7c, 0x0b, 0x1c, 0x61, 0x64, 0x8f, 0xe3, 0x23, - 0x95, 0x08, 0xec, 0x37, 0x58, 0x6e, 0xb2, 0x13, 0x49, 0xca, 0x3c, 0x85, 0xa6, 0x98, 0x30, 0x28, - 0x68, 0x94, 0xb3, 0x38, 0x34, 0x5a, 0x5a, 0xd5, 0x45, 0x13, 0x65, 0x6c, 0xbb, 0x92, 0xda, 0x36, - 0x7f, 0x46, 0xda, 0x28, 0x8a, 0x0b, 0xb5, 0x1d, 0xc0, 0x1a, 0x0d, 0xfa, 0x74, 0x62, 0xd4, 0x8f, - 0x1b, 0x22, 0x9d, 0x69, 0xe8, 0xad, 0x58, 0x1c, 0xeb, 0xa3, 0x28, 0xdc, 0xa5, 0x8f, 0x2e, 0x09, - 0x19, 0x7d, 0x69, 0xe8, 0xac, 0x58, 0x6c, 0xfe, 0x55, 0x83, 0x35, 0x52, 0x13, 0xb2, 0x27, 0x50, - 0x75, 0xd0, 0x6b, 0x54, 0x55, 0x3f, 0x6e, 0x61, 0x4d, 0xe5, 0x62, 0xf1, 0x79, 0xc9, 0x92, 0xab, - 0xc4, 0x7a, 0x82, 0x85, 0x51, 0x50, 0xd6, 0xab, 0xde, 0x8a, 0xf5, 0xb4, 0x4a, 0xac, 0x27, 0xb3, - 0x18, 0x21, 0x65, 0xbd, 0xea, 0x8d, 0x58, 0x4f, 0xab, 0x9e, 0xd5, 0xa0, 0x4a, 0xb5, 0x24, 0x2e, - 0x19, 0xa8, 0x37, 0xb3, 0x03, 0xdb, 0x19, 0xb8, 0xb5, 0x04, 0x56, 0x3b, 0x03, 0xab, 0x96, 0x98, - 0x6f, 0x67, 0xcc, 0xd7, 0x62, 0x33, 0xa2, 0x3c, 0x44, 0xfa, 0xe2, 0x6a, 0x24, 0xc2, 0xe4, 0xc0, - 0x54, 0x93, 0x4b, 0xb7, 0xbd, 0x0f, 0x61, 0x8d, 0xc0, 0x67, 0x86, 0x22, 0x19, 0x6a, 0x2b, 0x96, - 0x99, 0x7f, 0x2e, 0xa7, 0xbd, 0x7c, 0x70, 0xc9, 0xa7, 0xf6, 0xe2, 0x5e, 0x8e, 0xe2, 0xf4, 0x42, - 0x33, 0x37, 0x38, 0x2e, 0xbc, 0xd0, 0x88, 0x2d, 0x37, 0xb4, 0x23, 0xbb, 0x6f, 0x87, 0xc9, 0xb1, - 0x1b, 0xd3, 0xc2, 0xfb, 0xc8, 0xee, 0x3b, 0x5c, 0x9e, 0xba, 0x44, 0xe0, 0xe6, 0x40, 0x7b, 0x7a, - 0x55, 0x6e, 0x0e, 0xa4, 0xc4, 0xea, 0x91, 0x33, 0x0b, 0x2f, 0xf5, 0x35, 0xda, 0xd2, 0x48, 0x08, - 0x34, 0x62, 0x94, 0xd4, 0x6b, 0xc8, 0xc4, 0x6f, 0xb1, 0x95, 0x47, 0x81, 0x37, 0xa5, 0x63, 0x43, - 0x5f, 0xa7, 0x7b, 0x63, 0xca, 0x89, 0xe5, 0x17, 0x76, 0x30, 0xe6, 0x91, 0x0e, 0xa9, 0x9c, 0x38, - 0xea, 0xc9, 0x23, 0xe3, 0x72, 0x2f, 0x27, 0xcf, 0x21, 0xb4, 0x5e, 0xf0, 0xe8, 0x7c, 0xd6, 0x17, - 0x47, 0x73, 0x77, 0x34, 0xbe, 0xe3, 0xe0, 0x31, 0x5f, 0xc1, 0x4e, 0x6e, 0xed, 0xd2, 0x10, 0x19, - 0xac, 0x0c, 0x46, 0xe3, 0x38, 0x61, 0xf8, 0x6d, 0xf6, 0x60, 0xf3, 0x05, 0x8f, 0x14, 0xdb, 0x8f, - 0x95, 0xa3, 0x46, 0x0e, 0x86, 0xdd, 0xd1, 0xf8, 0xe2, 0xd6, 0xe7, 0x77, 0x9c, 0x3b, 0xa7, 0xd0, - 0x88, 0xb5, 0x2c, 0x8d, 0xaa, 0x09, 0x95, 0xc1, 0x28, 0x19, 0x29, 0x07, 0xa3, 0xb1, 0xb9, 0x03, - 0xdb, 0x2f, 0xb8, 0xdc, 0xd7, 0x29, 0x32, 0xf3, 0x00, 0xa3, 0xa5, 0xb0, 0xa5, 0x29, 0xa9, 0x40, - 0x4b, 0x15, 0xfc, 0x51, 0x03, 0xf6, 0xb9, 0xed, 0x0e, 0x1d, 0xfe, 0x3c, 0x08, 0xbc, 0x60, 0xe1, - 0x1c, 0x8d, 0xd2, 0xef, 0x55, 0xe4, 0x7b, 0xb0, 0xde, 0x9f, 0xb8, 0x8e, 0x37, 0xfe, 0xc2, 0x0b, - 0x65, 0x95, 0xa7, 0x0c, 0x2c, 0xd1, 0xd7, 0x4e, 0x72, 0x57, 0x12, 0xdf, 0x66, 0x08, 0xdb, 0x19, - 0x48, 0xf7, 0x52, 0x60, 0x2f, 0x60, 0xe7, 0x22, 0xb0, 0xdd, 0x70, 0xc4, 0x83, 0xec, 0xf0, 0x96, - 0x9e, 0x47, 0x9a, 0x7a, 0x1e, 0x29, 0x6d, 0x8b, 0x2c, 0x4b, 0x4a, 0x0c, 0x37, 0x79, 0x45, 0x4b, - 0x1f, 0xf0, 0xc3, 0xe4, 0xa1, 0x23, 0x33, 0xf0, 0x3f, 0x52, 0xb2, 0xb2, 0xa9, 0xdc, 0x43, 0xbe, - 0x3c, 0x8e, 0x07, 0x49, 0x89, 0xb4, 0xbc, 0x00, 0x29, 0xa5, 0x26, 0x46, 0x1a, 0x25, 0x2d, 0xee, - 0x1e, 0xa7, 0xf7, 0xc3, 0x3e, 0xd4, 0xe2, 0xf1, 0x97, 0x6d, 0xc3, 0x7b, 0x27, 0xee, 0xb5, 0xed, - 0x4c, 0x86, 0x31, 0xab, 0x59, 0x62, 0xef, 0x41, 0x1d, 0x5f, 0xae, 0x88, 0xd5, 0xd4, 0x58, 0x13, - 0x36, 0xe8, 0x89, 0x44, 0x72, 0xca, 0xac, 0x01, 0x70, 0x1e, 0x79, 0xbe, 0xa4, 0x2b, 0x48, 0x5f, - 0x7a, 0x37, 0x92, 0x5e, 0x39, 0xfc, 0x29, 0xd4, 0xe2, 0x99, 0x4b, 0xb1, 0x11, 0xb3, 0x9a, 0x25, - 0xb6, 0x05, 0x9b, 0xcf, 0xaf, 0x27, 0x83, 0x28, 0x61, 0x69, 0x6c, 0x17, 0xb6, 0xbb, 0xb6, 0x3b, - 0xe0, 0x4e, 0x56, 0x50, 0x3e, 0x74, 0x61, 0x4d, 0x6e, 0x6b, 0x01, 0x4d, 0xea, 0x12, 0x64, 0xb3, - 0xc4, 0x36, 0xa0, 0x26, 0x9a, 0x0c, 0x52, 0x9a, 0x80, 0x41, 0x7b, 0x0e, 0x69, 0x84, 0x49, 0x51, - 0x40, 0x9a, 0x60, 0x22, 0x44, 0xa4, 0x57, 0x58, 0x0b, 0x9a, 0xf8, 0x6b, 0x3e, 0xf5, 0x1d, 0x3b, - 0x22, 0xee, 0xea, 0x61, 0x0f, 0xd6, 0x93, 0xbc, 0x8a, 0x25, 0xd2, 0x62, 0xc2, 0x6b, 0x96, 0x44, - 0x44, 0x30, 0x44, 0xc8, 0xfb, 0xf2, 0xb8, 0xa9, 0x51, 0xd0, 0x3c, 0x3f, 0x66, 0x94, 0x8f, 0xff, - 0xd6, 0x80, 0x2a, 0x81, 0x61, 0x5f, 0xc1, 0x7a, 0xf2, 0x14, 0xc8, 0xf0, 0x70, 0xcf, 0xbf, 0x3d, - 0x1a, 0x3b, 0x39, 0x2e, 0x25, 0xcd, 0x7c, 0xfc, 0xdb, 0x7f, 0xfc, 0xfb, 0x4f, 0xe5, 0x07, 0x66, - 0xeb, 0xc8, 0xf6, 0x27, 0xe1, 0xd1, 0xf5, 0x53, 0xdb, 0xf1, 0x2f, 0xed, 0xa7, 0x47, 0x62, 0xcb, - 0x87, 0x1f, 0x6b, 0x87, 0x6c, 0x04, 0x75, 0xe5, 0x45, 0x8e, 0xb5, 0x85, 0x9a, 0xf9, 0x37, 0x40, - 0x63, 0x77, 0x8e, 0x2f, 0x0d, 0x7c, 0x84, 0x06, 0xf6, 0x8d, 0x87, 0x45, 0x06, 0x8e, 0xde, 0x88, - 0x8e, 0xf9, 0x8d, 0xb0, 0xf3, 0x09, 0x40, 0xfa, 0x4a, 0xc6, 0x10, 0xed, 0xdc, 0xc3, 0x9b, 0xd1, - 0xce, 0xb3, 0xa5, 0x91, 0x12, 0x73, 0xa0, 0xae, 0x3c, 0x28, 0x31, 0x23, 0xf7, 0xc2, 0xa4, 0xbc, - 0x80, 0x19, 0x0f, 0x0b, 0x65, 0x52, 0xd3, 0x07, 0x08, 0xb7, 0xc3, 0xf6, 0x72, 0x70, 0x43, 0x5c, - 0x2a, 0xf1, 0xb2, 0x2e, 0x6c, 0xa8, 0xef, 0x36, 0x0c, 0xbd, 0x2f, 0x78, 0xb0, 0x32, 0xf4, 0x79, - 0x41, 0x02, 0xf9, 0x33, 0xd8, 0xcc, 0xbc, 0x94, 0x30, 0x5c, 0x5c, 0xf4, 0x54, 0x63, 0x3c, 0x28, - 0x90, 0x24, 0x7a, 0xbe, 0x82, 0xf6, 0xfc, 0xcb, 0x06, 0x46, 0xf1, 0x91, 0x92, 0x94, 0xf9, 0xd7, - 0x05, 0xa3, 0xb3, 0x48, 0x9c, 0xa8, 0x3e, 0x83, 0x66, 0xfe, 0x05, 0x80, 0x61, 0xf8, 0x16, 0x3c, - 0x58, 0x18, 0x7b, 0xc5, 0xc2, 0x44, 0xe1, 0xc7, 0xb0, 0x9e, 0x5c, 0xbf, 0xa9, 0x50, 0xf3, 0xf7, - 0x7c, 0x2a, 0xd4, 0xb9, 0x3b, 0xba, 0x59, 0x62, 0x63, 0xd8, 0xcc, 0xdc, 0x88, 0x29, 0x5e, 0x45, - 0xd7, 0x71, 0x8a, 0x57, 0xe1, 0xf5, 0xd9, 0x7c, 0x1f, 0x13, 0xfc, 0xd0, 0x68, 0xe7, 0x13, 0x4c, - 0xcd, 0x4b, 0x94, 0xe2, 0x09, 0x34, 0xb2, 0x97, 0x57, 0xf6, 0x80, 0x5a, 0x71, 0xc1, 0xbd, 0xd8, - 0x30, 0x8a, 0x44, 0x09, 0xe6, 0x00, 0x36, 0x33, 0x77, 0x50, 0x89, 0xb9, 0xe0, 0x5a, 0x2b, 0x31, - 0x17, 0x5d, 0x58, 0xcd, 0x1f, 0x22, 0xe6, 0x8f, 0x0e, 0x3f, 0xc8, 0x61, 0x96, 0xa3, 0xec, 0xd1, - 0x1b, 0x31, 0x8b, 0x7c, 0x13, 0x17, 0xe7, 0x55, 0x12, 0x27, 0x6a, 0x71, 0x99, 0x38, 0x65, 0xee, - 0xb1, 0x99, 0x38, 0x65, 0xef, 0xaa, 0xe6, 0x87, 0x68, 0xf3, 0xb1, 0x61, 0xe4, 0x6c, 0xd2, 0xa8, - 0x7f, 0xf4, 0xc6, 0xf3, 0x71, 0xdb, 0xfe, 0x0a, 0x20, 0x1d, 0xd6, 0x69, 0xdb, 0xce, 0xdd, 0x17, - 0x68, 0xdb, 0xce, 0xcf, 0xf4, 0x66, 0x07, 0x6d, 0xe8, 0xac, 0x5d, 0xec, 0x17, 0x1b, 0xa5, 0x19, - 0xa7, 0x21, 0x38, 0x93, 0x71, 0x75, 0x68, 0xcf, 0x66, 0x3c, 0x33, 0xb6, 0x9a, 0xfb, 0x68, 0xc5, - 0x30, 0x76, 0xf2, 0x19, 0xc7, 0x65, 0xc2, 0x09, 0x07, 0xe7, 0xbe, 0x74, 0x9c, 0x24, 0x3b, 0x45, - 0xd3, 0x28, 0xd9, 0x29, 0x9c, 0x3d, 0xe3, 0x4e, 0xc7, 0x3a, 0x79, 0x3b, 0xb3, 0xbe, 0xda, 0xec, - 0xd8, 0x05, 0x54, 0x69, 0x3e, 0x64, 0x5b, 0x52, 0x99, 0xa2, 0x9f, 0xa9, 0x2c, 0xa9, 0xf8, 0x07, - 0xa8, 0xf8, 0x11, 0xbb, 0xab, 0x85, 0xb2, 0xaf, 0xa1, 0xae, 0x8c, 0x54, 0xd4, 0xa7, 0xe7, 0xc7, - 0x3e, 0xea, 0xd3, 0x05, 0xb3, 0xd7, 0xc2, 0x28, 0x71, 0xb1, 0x0a, 0xb7, 0x45, 0x17, 0x36, 0xd4, - 0x91, 0x93, 0x9a, 0x5e, 0xc1, 0x6c, 0x6a, 0xe8, 0xf3, 0x82, 0x64, 0x43, 0x9c, 0x40, 0x23, 0x3b, - 0x3b, 0xd1, 0xde, 0x2a, 0x1c, 0xcc, 0x68, 0x6f, 0x15, 0x8f, 0x5a, 0x66, 0x49, 0xe0, 0x51, 0x87, - 0x1b, 0xa6, 0x1e, 0x41, 0x99, 0xa6, 0xa4, 0xcf, 0x0b, 0x62, 0x25, 0xcf, 0xf4, 0xbf, 0xbf, 0xed, - 0x68, 0xdf, 0xbd, 0xed, 0x68, 0xff, 0x7a, 0xdb, 0xd1, 0xfe, 0xf0, 0xae, 0x53, 0xfa, 0xee, 0x5d, - 0xa7, 0xf4, 0xcf, 0x77, 0x9d, 0x52, 0xbf, 0x8a, 0xff, 0xe3, 0xfd, 0xe8, 0x3f, 0x01, 0x00, 0x00, - 0xff, 0xff, 0xa2, 0xd2, 0xac, 0xe5, 0x0b, 0x1c, 0x00, 0x00, + // 2101 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x59, 0xcd, 0x6f, 0xe3, 0xc6, + 0x15, 0x17, 0x25, 0x59, 0x96, 0x9e, 0x6c, 0x45, 0x1e, 0xcb, 0x32, 0x97, 0xeb, 0x68, 0x1d, 0x36, + 0x09, 0x0c, 0xa3, 0x58, 0x63, 0xdd, 0x9e, 0x02, 0xa4, 0x40, 0x56, 0xda, 0x6c, 0x8c, 0x7a, 0xe3, + 0x94, 0xf6, 0xa6, 0x08, 0x0a, 0x14, 0xa5, 0xa4, 0x91, 0x2c, 0x98, 0x22, 0xb9, 0x24, 0x65, 0xd7, + 0x58, 0xa4, 0x87, 0x9e, 0x7a, 0xea, 0x07, 0x0a, 0x34, 0xc7, 0x1e, 0xfa, 0x9f, 0xf4, 0xd4, 0x63, + 0x80, 0x5e, 0x7a, 0x2c, 0x76, 0xfb, 0x87, 0x14, 0xf3, 0x66, 0x86, 0x1c, 0x7e, 0xc8, 0xa9, 0x02, + 0xd4, 0x37, 0xbe, 0xf7, 0x46, 0xef, 0xfd, 0xde, 0xc7, 0xbc, 0x79, 0x33, 0x82, 0xd6, 0x78, 0x3e, + 0xb7, 0xc3, 0x88, 0x06, 0x8f, 0xfd, 0xc0, 0x8b, 0x3c, 0x52, 0xf6, 0x87, 0x46, 0x6b, 0x3c, 0xbf, + 0xf1, 0x82, 0x2b, 0xc9, 0x33, 0xf6, 0xa6, 0x9e, 0x37, 0x75, 0xe8, 0x91, 0xed, 0xcf, 0x8e, 0x6c, + 0xd7, 0xf5, 0x22, 0x3b, 0x9a, 0x79, 0x6e, 0xc8, 0xa5, 0xe6, 0x6f, 0xa0, 0x7d, 0x1e, 0xd9, 0x41, + 0x74, 0x61, 0x87, 0x57, 0x16, 0x7d, 0xb5, 0xa0, 0x61, 0x44, 0x08, 0x54, 0x23, 0x3b, 0xbc, 0xd2, + 0xb5, 0x7d, 0xed, 0xa0, 0x61, 0xe1, 0x37, 0xd1, 0x61, 0x3d, 0xf4, 0x16, 0xc1, 0x88, 0x86, 0x7a, + 0x79, 0xbf, 0x72, 0xd0, 0xb0, 0x24, 0x49, 0x7a, 0x00, 0x01, 0x9d, 0x7b, 0xd7, 0xf4, 0x05, 0x8d, + 0x6c, 0xbd, 0xb2, 0xaf, 0x1d, 0xd4, 0x2d, 0x85, 0x43, 0xf6, 0xa0, 0x11, 0xa2, 0x85, 0xd9, 0x9c, + 0xea, 0x55, 0x54, 0x99, 0x30, 0xcc, 0x57, 0xb0, 0xa5, 0xd8, 0x0f, 0x7d, 0xcf, 0x0d, 0x29, 0xe9, + 0x42, 0x2d, 0xa0, 0xe1, 0xc2, 0x89, 0x10, 0x42, 0xdd, 0x12, 0x14, 0x69, 0x43, 0x65, 0x1e, 0x4e, + 0xf5, 0x32, 0x2a, 0x61, 0x9f, 0xe4, 0x38, 0x81, 0x55, 0xd9, 0xaf, 0x1c, 0x34, 0x8f, 0xf5, 0xc7, + 0xfe, 0xf0, 0x71, 0xdf, 0x9b, 0xcf, 0x3d, 0xf7, 0xe7, 0x18, 0x05, 0xa9, 0x34, 0x06, 0x6c, 0xfe, + 0x12, 0xc8, 0x99, 0x4f, 0x03, 0x3b, 0xa2, 0xaa, 0xd3, 0x06, 0x94, 0x3d, 0x1f, 0xed, 0xb5, 0x8e, + 0x81, 0x29, 0x61, 0xc2, 0x33, 0xdf, 0x2a, 0x7b, 0x3e, 0x0b, 0x88, 0x6b, 0xcf, 0xa9, 0x30, 0x8c, + 0xdf, 0x6a, 0x40, 0x2a, 0xa9, 0x80, 0x98, 0x7f, 0xd0, 0x60, 0x3b, 0x65, 0x40, 0x78, 0x75, 0x97, + 0x85, 0xc4, 0xe3, 0x72, 0x91, 0xc7, 0x95, 0x42, 0x8f, 0xab, 0xff, 0xab, 0xc7, 0x9f, 0xc0, 0xd6, + 0x4b, 0x7f, 0x9c, 0x71, 0x78, 0xa5, 0x2c, 0x9b, 0x01, 0x10, 0x55, 0xc5, 0xbd, 0x24, 0xea, 0x53, + 0xe8, 0xfe, 0x6c, 0x41, 0x83, 0xdb, 0xf3, 0xc8, 0x8e, 0x16, 0xe1, 0xe9, 0x2c, 0x8c, 0x14, 0xec, + 0x98, 0x10, 0xad, 0x38, 0x21, 0x19, 0xec, 0xd7, 0xb0, 0x9b, 0xd3, 0xb3, 0xb2, 0x03, 0x4f, 0xb2, + 0x0e, 0xec, 0x32, 0x07, 0x14, 0xbd, 0x79, 0xfc, 0x7d, 0xd8, 0x3e, 0xbf, 0xf4, 0x6e, 0x06, 0x83, + 0xd3, 0x53, 0x6f, 0x74, 0x15, 0x7e, 0xbf, 0xc0, 0xff, 0x55, 0x83, 0x75, 0xa1, 0x81, 0xb4, 0xa0, + 0x7c, 0x32, 0x10, 0xbf, 0x2b, 0x9f, 0x0c, 0x62, 0x4d, 0x65, 0x45, 0x13, 0x81, 0xea, 0xdc, 0x1b, + 0x53, 0x51, 0x32, 0xf8, 0x4d, 0x3a, 0xb0, 0xe6, 0xdd, 0xb8, 0x34, 0x10, 0xdb, 0x8f, 0x13, 0x6c, + 0xe5, 0x60, 0x70, 0x1a, 0xea, 0x6b, 0x68, 0x10, 0xbf, 0x59, 0x3c, 0xc2, 0x5b, 0x77, 0x44, 0xc7, + 0x7a, 0x0d, 0xb9, 0x82, 0x22, 0x06, 0xd4, 0x17, 0xae, 0x90, 0xac, 0xa3, 0x24, 0xa6, 0xcd, 0x11, + 0x74, 0xd2, 0x6e, 0xae, 0x1c, 0xdb, 0xf7, 0x60, 0xcd, 0x61, 0x3f, 0x15, 0x91, 0x6d, 0xb2, 0xc8, + 0x0a, 0x75, 0x16, 0x97, 0x98, 0x0e, 0x74, 0x5e, 0xba, 0xec, 0x53, 0xf2, 0x45, 0x30, 0xb3, 0x21, + 0x31, 0x61, 0x23, 0xa0, 0xbe, 0x63, 0x8f, 0xe8, 0x19, 0x7a, 0xcc, 0xad, 0xa4, 0x78, 0x64, 0x1f, + 0x9a, 0x13, 0x2f, 0x18, 0x51, 0x0b, 0x9b, 0x94, 0x68, 0x59, 0x2a, 0xcb, 0xfc, 0x04, 0x76, 0x32, + 0xd6, 0x56, 0xf5, 0xc9, 0xb4, 0xe0, 0x81, 0x68, 0x02, 0xb2, 0xbc, 0x1d, 0xfb, 0x56, 0xa2, 0x7e, + 0xa8, 0xb4, 0x02, 0xf4, 0x16, 0xa5, 0xa2, 0x17, 0x2c, 0xaf, 0x85, 0x6f, 0x34, 0x30, 0x8a, 0x94, + 0x0a, 0x70, 0x77, 0x6a, 0xfd, 0xff, 0x76, 0x98, 0x6f, 0x34, 0xd8, 0xfd, 0x62, 0x11, 0x4c, 0x8b, + 0x9c, 0x55, 0xfc, 0xd1, 0xd2, 0x47, 0x87, 0x01, 0xf5, 0x99, 0x6b, 0x8f, 0xa2, 0xd9, 0x35, 0x15, + 0xa8, 0x62, 0x1a, 0x6b, 0x9b, 0x9d, 0x18, 0x0c, 0x58, 0xc5, 0xc2, 0x6f, 0xb6, 0x7e, 0x32, 0x73, + 0x28, 0x6e, 0x7d, 0x5e, 0xca, 0x31, 0x8d, 0x95, 0xbb, 0x18, 0x0e, 0x66, 0x81, 0xbe, 0x86, 0x12, + 0x41, 0x99, 0xbf, 0x06, 0x3d, 0x0f, 0xec, 0x5e, 0xda, 0xd7, 0x35, 0xb4, 0xfb, 0x97, 0x74, 0x74, + 0xf5, 0x5d, 0x4d, 0xb7, 0x0b, 0x35, 0x1a, 0x04, 0x7d, 0x97, 0x67, 0xa6, 0x62, 0x09, 0x8a, 0xc5, + 0xed, 0xc6, 0x0e, 0x5c, 0x26, 0xe0, 0x41, 0x90, 0xe4, 0x77, 0x1c, 0xa9, 0x1f, 0xc3, 0x96, 0x62, + 0x77, 0xe5, 0xc2, 0xfd, 0x9d, 0x06, 0x1d, 0x51, 0x64, 0xe7, 0xe8, 0x89, 0xc4, 0xbe, 0xa7, 0x94, + 0xd7, 0x06, 0x73, 0x9f, 0x8b, 0x93, 0xfa, 0x1a, 0x79, 0xee, 0x64, 0x36, 0x15, 0x45, 0x2b, 0x28, + 0x96, 0x33, 0x1e, 0x90, 0x93, 0x81, 0x38, 0x28, 0x63, 0x9a, 0x8d, 0x0e, 0x7c, 0x54, 0xf9, 0x3c, + 0xc9, 0xa8, 0xc2, 0x31, 0x17, 0xb0, 0x93, 0x41, 0x72, 0x2f, 0x89, 0x7b, 0x06, 0x3b, 0x16, 0x9d, + 0xce, 0xd8, 0x5c, 0x25, 0x97, 0xdc, 0x79, 0xec, 0xd8, 0xe3, 0x71, 0x40, 0xc3, 0x50, 0x98, 0x95, + 0xa4, 0xf9, 0x14, 0xba, 0x59, 0x35, 0x2b, 0x27, 0xe3, 0x27, 0xd0, 0x39, 0x9b, 0x4c, 0x9c, 0x99, + 0x4b, 0x5f, 0xd0, 0xf9, 0x30, 0x85, 0x24, 0xba, 0xf5, 0x63, 0x24, 0xec, 0xbb, 0x68, 0x4a, 0x61, + 0x8d, 0x2c, 0xf3, 0xfb, 0x95, 0x21, 0xfc, 0x38, 0x2e, 0x87, 0x53, 0x6a, 0x8f, 0x13, 0x08, 0xb9, + 0x72, 0xe0, 0x62, 0x5e, 0x0e, 0x68, 0x38, 0xfd, 0xab, 0x95, 0x0d, 0xff, 0x5e, 0x03, 0x78, 0x81, + 0xd3, 0xed, 0x89, 0x3b, 0xf1, 0x0a, 0x83, 0x6f, 0x40, 0x7d, 0x8e, 0x7e, 0x9d, 0x0c, 0xf0, 0x97, + 0x55, 0x2b, 0xa6, 0xd9, 0xa1, 0x67, 0x3b, 0xb3, 0xb8, 0xbf, 0x73, 0x82, 0xfd, 0xc2, 0xa7, 0x34, + 0x78, 0x69, 0x9d, 0xf2, 0xee, 0xd6, 0xb0, 0x62, 0x9a, 0x95, 0xe3, 0xc8, 0x99, 0x51, 0x37, 0x42, + 0x29, 0x3f, 0x16, 0x15, 0x8e, 0x39, 0x04, 0xe0, 0x89, 0x5c, 0x8a, 0x87, 0x40, 0x95, 0x65, 0x5f, + 0xa6, 0x80, 0x7d, 0x33, 0x1c, 0x61, 0x64, 0x4f, 0xe5, 0x89, 0xcc, 0x09, 0x6c, 0x57, 0x58, 0x6e, + 0xa2, 0xec, 0x05, 0x65, 0x9e, 0x42, 0x9b, 0x0d, 0x28, 0x3c, 0x68, 0x3c, 0x67, 0x32, 0x34, 0x5a, + 0x52, 0xd5, 0x45, 0x03, 0xa9, 0xb4, 0x5d, 0x49, 0x6c, 0x9b, 0x9f, 0x73, 0x6d, 0x3c, 0x8a, 0x4b, + 0xb5, 0x1d, 0xc0, 0x3a, 0xbf, 0x45, 0xf0, 0x03, 0xa7, 0x79, 0xdc, 0x62, 0xe9, 0x4c, 0x42, 0x6f, + 0x49, 0xb1, 0xd4, 0xc7, 0xa3, 0x70, 0x97, 0x3e, 0xbe, 0x89, 0x53, 0xfa, 0x92, 0xd0, 0x59, 0x52, + 0x6c, 0xfe, 0x4d, 0x83, 0x75, 0xae, 0x26, 0x24, 0x8f, 0xa1, 0xe6, 0xa0, 0xd7, 0xa8, 0xaa, 0x79, + 0xdc, 0xc1, 0x9a, 0xca, 0xc4, 0xe2, 0xb3, 0x92, 0x25, 0x56, 0xb1, 0xf5, 0x1c, 0x16, 0x46, 0x41, + 0x59, 0xaf, 0x7a, 0xcb, 0xd6, 0xf3, 0x55, 0x6c, 0x3d, 0x37, 0x8b, 0x11, 0x52, 0xd6, 0xab, 0xde, + 0xb0, 0xf5, 0x7c, 0xd5, 0xd3, 0x3a, 0xd4, 0x78, 0x2d, 0xb1, 0x3b, 0x0a, 0xea, 0x4d, 0xed, 0xc0, + 0x6e, 0x0a, 0x6e, 0x3d, 0x86, 0xd5, 0x4d, 0xc1, 0xaa, 0xc7, 0xe6, 0xbb, 0x29, 0xf3, 0x75, 0x69, + 0x86, 0x95, 0x07, 0x4b, 0x9f, 0xac, 0x46, 0x4e, 0x98, 0x14, 0x88, 0x6a, 0x72, 0xe5, 0xb6, 0xf7, + 0x01, 0xac, 0x73, 0xf0, 0xa9, 0x99, 0x4a, 0x84, 0xda, 0x92, 0x32, 0xf3, 0x2f, 0xe5, 0xa4, 0xd7, + 0x8f, 0x2e, 0xe9, 0xdc, 0x5e, 0xde, 0xeb, 0x51, 0x9c, 0xdc, 0x87, 0x72, 0x73, 0xe7, 0xd2, 0xfb, + 0x10, 0xdb, 0x72, 0x63, 0x3b, 0xb2, 0x87, 0x76, 0x18, 0x9f, 0xda, 0x92, 0x66, 0xde, 0x47, 0xf6, + 0xd0, 0xa1, 0xe2, 0xd0, 0xe6, 0x04, 0x6e, 0x0e, 0xb4, 0xa7, 0xd7, 0xc4, 0xe6, 0x40, 0x8a, 0xad, + 0x9e, 0x38, 0x8b, 0xf0, 0x52, 0x5f, 0xe7, 0x5b, 0x1a, 0x09, 0x86, 0x86, 0x4d, 0xa2, 0x7a, 0x1d, + 0x99, 0xf8, 0xcd, 0xb6, 0xf2, 0x24, 0xf0, 0xe6, 0xfc, 0xd8, 0xd0, 0x1b, 0xfc, 0x52, 0x9a, 0x70, + 0xa4, 0xfc, 0xc2, 0x0e, 0xa6, 0x34, 0xd2, 0x21, 0x91, 0x73, 0x8e, 0x7a, 0xf2, 0x88, 0xb8, 0xdc, + 0xcb, 0xc9, 0x73, 0x08, 0x9d, 0xe7, 0x34, 0x3a, 0x5f, 0x0c, 0xd9, 0xd9, 0xdd, 0x9f, 0x4c, 0xef, + 0x38, 0x78, 0xcc, 0x97, 0xb0, 0x93, 0x59, 0xbb, 0x32, 0x44, 0x02, 0xd5, 0xd1, 0x64, 0x2a, 0x13, + 0x86, 0xdf, 0xe6, 0x00, 0x36, 0x9f, 0xd3, 0x48, 0xb1, 0xfd, 0x48, 0x39, 0x6a, 0xc4, 0x5c, 0xd9, + 0x9f, 0x4c, 0x2f, 0x6e, 0x7d, 0x7a, 0xc7, 0xb9, 0x73, 0x0a, 0x2d, 0xa9, 0x65, 0x65, 0x54, 0x6d, + 0xa8, 0x8c, 0x26, 0xf1, 0x44, 0x3a, 0x9a, 0x4c, 0xcd, 0x1d, 0xd8, 0x7e, 0x4e, 0xc5, 0xbe, 0x4e, + 0x90, 0x99, 0x07, 0x18, 0x2d, 0x85, 0x2d, 0x4c, 0x09, 0x05, 0x5a, 0xa2, 0xe0, 0x4f, 0x1a, 0x90, + 0xcf, 0x6c, 0x77, 0xec, 0xd0, 0x67, 0x41, 0xe0, 0x05, 0x4b, 0xc7, 0x70, 0x94, 0x7e, 0xaf, 0x22, + 0xdf, 0x83, 0xc6, 0x70, 0xe6, 0x3a, 0xde, 0xf4, 0x0b, 0x2f, 0x94, 0x23, 0x59, 0xcc, 0xc0, 0x12, + 0x7d, 0xe5, 0xc4, 0x57, 0x2d, 0xf6, 0x6d, 0x86, 0xb0, 0x9d, 0x82, 0x74, 0x2f, 0x05, 0xf6, 0x1c, + 0x76, 0x2e, 0x02, 0xdb, 0x0d, 0x27, 0x34, 0x48, 0x0f, 0x77, 0xc9, 0x79, 0xa4, 0xa9, 0xe7, 0x91, + 0xd2, 0xb6, 0xb8, 0x65, 0x41, 0xb1, 0xe1, 0x26, 0xab, 0x68, 0xe5, 0x03, 0x7e, 0x1c, 0xbf, 0x93, + 0xa4, 0xee, 0x0b, 0xef, 0x2a, 0x59, 0xd9, 0x54, 0xae, 0x31, 0x5f, 0x1e, 0xcb, 0x41, 0x53, 0x20, + 0x2d, 0x2f, 0x41, 0xca, 0x53, 0x23, 0x91, 0x46, 0x71, 0x8b, 0xbb, 0xc7, 0xe1, 0xff, 0x70, 0x08, + 0x75, 0x39, 0x1e, 0x93, 0x6d, 0x78, 0xe7, 0xc4, 0xbd, 0xb6, 0x9d, 0xd9, 0x58, 0xb2, 0xda, 0x25, + 0xf2, 0x0e, 0x34, 0xf1, 0xe1, 0x8b, 0xb3, 0xda, 0x1a, 0x69, 0xc3, 0x06, 0x7f, 0x61, 0x11, 0x9c, + 0x32, 0x69, 0x01, 0x9c, 0x47, 0x9e, 0x2f, 0xe8, 0x0a, 0xd2, 0x97, 0xde, 0x8d, 0xa0, 0xab, 0x87, + 0x3f, 0x85, 0xba, 0x9c, 0xb9, 0x14, 0x1b, 0x92, 0xd5, 0x2e, 0x91, 0x2d, 0xd8, 0x7c, 0x76, 0x3d, + 0x1b, 0x45, 0x31, 0x4b, 0x23, 0xbb, 0xb0, 0xdd, 0xb7, 0xdd, 0x11, 0x75, 0xd2, 0x82, 0xf2, 0xa1, + 0x0b, 0xeb, 0x62, 0x5b, 0x33, 0x68, 0x42, 0x17, 0x23, 0xdb, 0x25, 0xb2, 0x01, 0x75, 0xd6, 0x64, + 0x90, 0xd2, 0x18, 0x0c, 0xbe, 0xe7, 0x90, 0x46, 0x98, 0x3c, 0x0a, 0x48, 0x73, 0x98, 0x08, 0x11, + 0xe9, 0x2a, 0xe9, 0x40, 0x1b, 0x7f, 0x4d, 0xe7, 0xbe, 0x63, 0x47, 0x9c, 0xbb, 0x76, 0x38, 0x80, + 0x46, 0x9c, 0x57, 0xb6, 0x44, 0x58, 0x8c, 0x79, 0xed, 0x12, 0x8b, 0x08, 0x86, 0x08, 0x79, 0x5f, + 0x1e, 0xb7, 0x35, 0x1e, 0x34, 0xcf, 0x97, 0x8c, 0xf2, 0xf1, 0xdf, 0x5b, 0x50, 0xe3, 0x60, 0xc8, + 0x57, 0xd0, 0x88, 0x5f, 0x12, 0x09, 0x1e, 0xee, 0xd9, 0x87, 0x4d, 0x63, 0x27, 0xc3, 0xe5, 0x49, + 0x33, 0x1f, 0xfd, 0xf6, 0x9f, 0xff, 0xf9, 0x73, 0xf9, 0x81, 0xd9, 0x39, 0xb2, 0xfd, 0x59, 0x78, + 0x74, 0xfd, 0xc4, 0x76, 0xfc, 0x4b, 0xfb, 0xc9, 0x11, 0xdb, 0xf2, 0xe1, 0x47, 0xda, 0x21, 0x99, + 0x40, 0x53, 0x79, 0xd0, 0x23, 0x5d, 0xa6, 0x26, 0xff, 0x84, 0x68, 0xec, 0xe6, 0xf8, 0xc2, 0xc0, + 0x87, 0x68, 0x60, 0xdf, 0x78, 0x58, 0x64, 0xe0, 0xe8, 0x35, 0xeb, 0x98, 0x5f, 0x33, 0x3b, 0x1f, + 0x03, 0x24, 0x8f, 0x6c, 0x04, 0xd1, 0xe6, 0xde, 0xed, 0x8c, 0x6e, 0x96, 0x2d, 0x8c, 0x94, 0x88, + 0x03, 0x4d, 0xe5, 0x3d, 0x8a, 0x18, 0x99, 0x07, 0x2a, 0xe5, 0x01, 0xcd, 0x78, 0x58, 0x28, 0x13, + 0x9a, 0xde, 0x47, 0xb8, 0x3d, 0xb2, 0x97, 0x81, 0x1b, 0xe2, 0x52, 0x81, 0x97, 0xf4, 0x61, 0x43, + 0x7d, 0xf6, 0x21, 0xe8, 0x7d, 0xc1, 0x7b, 0x97, 0xa1, 0xe7, 0x05, 0x31, 0xe4, 0x4f, 0x61, 0x33, + 0xf5, 0xd0, 0x42, 0x70, 0x71, 0xd1, 0x4b, 0x8f, 0xf1, 0xa0, 0x40, 0x12, 0xeb, 0xf9, 0x0a, 0xba, + 0xf9, 0x87, 0x11, 0x8c, 0xe2, 0xbb, 0x4a, 0x52, 0xf2, 0x8f, 0x13, 0x46, 0x6f, 0x99, 0x38, 0x56, + 0x7d, 0x06, 0xed, 0xec, 0x03, 0x02, 0xc1, 0xf0, 0x2d, 0x79, 0xef, 0x30, 0xf6, 0x8a, 0x85, 0xb1, + 0xc2, 0x8f, 0xa0, 0x11, 0xdf, 0xcf, 0x79, 0xa1, 0x66, 0x9f, 0x09, 0x78, 0xa1, 0xe6, 0x2e, 0xf1, + 0x66, 0x89, 0x4c, 0x61, 0x33, 0x75, 0x23, 0xe6, 0xf1, 0x2a, 0xba, 0xae, 0xf3, 0x78, 0x15, 0x5e, + 0x9f, 0xcd, 0xf7, 0x30, 0xc1, 0x0f, 0x8d, 0x6e, 0x36, 0xc1, 0xbc, 0x79, 0xb1, 0x52, 0x3c, 0x81, + 0x56, 0xfa, 0xf2, 0x4a, 0x1e, 0xf0, 0x56, 0x5c, 0x70, 0x2f, 0x36, 0x8c, 0x22, 0x51, 0x8c, 0x39, + 0x80, 0xcd, 0xd4, 0x1d, 0x54, 0x60, 0x2e, 0xb8, 0xd6, 0x0a, 0xcc, 0x45, 0x17, 0x56, 0xf3, 0x87, + 0x88, 0xf9, 0xc3, 0xc3, 0xf7, 0x33, 0x98, 0xc5, 0x28, 0x7b, 0xf4, 0x9a, 0xcd, 0x22, 0x5f, 0xcb, + 0xe2, 0xbc, 0x8a, 0xe3, 0xc4, 0x5b, 0x5c, 0x2a, 0x4e, 0xa9, 0x7b, 0x6c, 0x2a, 0x4e, 0xe9, 0xbb, + 0xaa, 0xf9, 0x01, 0xda, 0x7c, 0x64, 0x18, 0x19, 0x9b, 0x7c, 0xd4, 0x3f, 0x7a, 0xed, 0xf9, 0xb8, + 0x6d, 0x7f, 0x01, 0x90, 0x0c, 0xeb, 0x7c, 0xdb, 0xe6, 0xee, 0x0b, 0x7c, 0xdb, 0xe6, 0x67, 0x7a, + 0xb3, 0x87, 0x36, 0x74, 0xd2, 0x2d, 0xf6, 0x8b, 0x4c, 0x92, 0x8c, 0xf3, 0x21, 0x38, 0x95, 0x71, + 0x75, 0x68, 0x4f, 0x67, 0x3c, 0x35, 0xb6, 0x9a, 0xfb, 0x68, 0xc5, 0x30, 0x76, 0xb2, 0x19, 0xc7, + 0x65, 0xcc, 0x09, 0x07, 0xe7, 0xbe, 0x64, 0x9c, 0xe4, 0x76, 0x8a, 0xa6, 0x51, 0x6e, 0xa7, 0x70, + 0xf6, 0x94, 0x9d, 0x8e, 0xf4, 0xb2, 0x76, 0x16, 0x43, 0xb5, 0xd9, 0x91, 0x0b, 0xa8, 0xf1, 0xf9, + 0x90, 0x6c, 0x09, 0x65, 0x8a, 0x7e, 0xa2, 0xb2, 0x84, 0xe2, 0x1f, 0xa0, 0xe2, 0x77, 0xc9, 0x5d, + 0x2d, 0x94, 0xfc, 0x0a, 0x9a, 0xca, 0x48, 0xc5, 0xfb, 0x74, 0x7e, 0xec, 0xe3, 0x7d, 0xba, 0x60, + 0xf6, 0x5a, 0x1a, 0x25, 0xca, 0x56, 0xe1, 0xb6, 0xe8, 0xc3, 0x86, 0x3a, 0x72, 0xf2, 0xa6, 0x57, + 0x30, 0x9b, 0x1a, 0x7a, 0x5e, 0x10, 0x6f, 0x88, 0x13, 0x68, 0xa5, 0x67, 0x27, 0xbe, 0xb7, 0x0a, + 0x07, 0x33, 0xbe, 0xb7, 0x8a, 0x47, 0x2d, 0xb3, 0xc4, 0xf0, 0xa8, 0xc3, 0x0d, 0x51, 0x8f, 0xa0, + 0x54, 0x53, 0xd2, 0xf3, 0x02, 0xa9, 0xe4, 0xa9, 0xfe, 0x8f, 0x37, 0x3d, 0xed, 0xdb, 0x37, 0x3d, + 0xed, 0xdf, 0x6f, 0x7a, 0xda, 0x1f, 0xdf, 0xf6, 0x4a, 0xdf, 0xbe, 0xed, 0x95, 0xfe, 0xf5, 0xb6, + 0x57, 0x1a, 0xd6, 0xf0, 0x4f, 0xc2, 0x1f, 0xfd, 0x37, 0x00, 0x00, 0xff, 0xff, 0xab, 0x51, 0x94, + 0xcf, 0x68, 0x1c, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -4169,6 +4195,13 @@ func (m *StartTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.StartTime) > 0 { + i -= len(m.StartTime) + copy(dAtA[i:], m.StartTime) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.StartTime))) + i-- + dAtA[i] = 0x22 + } if m.RemoveMeta { i-- if m.RemoveMeta { @@ -5030,6 +5063,13 @@ func (m *CheckTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.StartTime) > 0 { + i -= len(m.StartTime) + copy(dAtA[i:], m.StartTime) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.StartTime))) + i-- + dAtA[i] = 0x22 + } if m.WarnCnt != 0 { i = encodeVarintDmmaster(dAtA, i, uint64(m.WarnCnt)) i-- @@ -5110,6 +5150,13 @@ func (m *OperateSourceRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.WorkerName) > 0 { + i -= len(m.WorkerName) + copy(dAtA[i:], m.WorkerName) + i = encodeVarintDmmaster(dAtA, i, uint64(len(m.WorkerName))) + i-- + dAtA[i] = 0x22 + } if len(m.SourceID) > 0 { for iNdEx := len(m.SourceID) - 1; iNdEx >= 0; iNdEx-- { i -= len(m.SourceID[iNdEx]) @@ -6561,6 +6608,10 @@ func (m *StartTaskRequest) Size() (n int) { if m.RemoveMeta { n += 2 } + l = len(m.StartTime) + if l > 0 { + n += 1 + l + sovDmmaster(uint64(l)) + } return n } @@ -6945,6 +6996,10 @@ func (m *CheckTaskRequest) Size() (n int) { if m.WarnCnt != 0 { n += 1 + sovDmmaster(uint64(m.WarnCnt)) } + l = len(m.StartTime) + if l > 0 { + n += 1 + l + sovDmmaster(uint64(l)) + } return n } @@ -6985,6 +7040,10 @@ func (m *OperateSourceRequest) Size() (n int) { n += 1 + l + sovDmmaster(uint64(l)) } } + l = len(m.WorkerName) + if l > 0 { + n += 1 + l + sovDmmaster(uint64(l)) + } return n } @@ -7724,6 +7783,38 @@ func (m *StartTaskRequest) Unmarshal(dAtA []byte) error { } } m.RemoveMeta = bool(v != 0) + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StartTime", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmmaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthDmmaster + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StartTime = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipDmmaster(dAtA[iNdEx:]) @@ -10105,6 +10196,38 @@ func (m *CheckTaskRequest) Unmarshal(dAtA []byte) error { break } } + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StartTime", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmmaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthDmmaster + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StartTime = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipDmmaster(dAtA[iNdEx:]) @@ -10340,6 +10463,38 @@ func (m *OperateSourceRequest) Unmarshal(dAtA []byte) error { } m.SourceID = append(m.SourceID, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WorkerName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmmaster + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthDmmaster + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthDmmaster + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.WorkerName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipDmmaster(dAtA[iNdEx:]) diff --git a/dm/dm/proto/dmmaster.proto b/dm/dm/proto/dmmaster.proto index a4504375923..0ba158c8b8f 100644 --- a/dm/dm/proto/dmmaster.proto +++ b/dm/dm/proto/dmmaster.proto @@ -6,133 +6,134 @@ import "dmworker.proto"; // refine if needed import "google/api/annotations.proto"; service Master { - rpc StartTask (StartTaskRequest) returns (StartTaskResponse) { - option (google.api.http) = { - post: "/apis/v1alpha1/tasks" - body: "*" - }; - } - - rpc OperateTask (OperateTaskRequest) returns (OperateTaskResponse) { - option (google.api.http) = { - put: "/apis/v1alpha1/tasks/{name}" - body: "*" - }; - } - rpc UpdateTask (UpdateTaskRequest) returns (UpdateTaskResponse) {} - - rpc QueryStatus (QueryStatusListRequest) returns (QueryStatusListResponse) { - option (google.api.http) = { - get: "/apis/v1alpha1/status/{name}" - }; - } - - // show un-resolved DDL locks - rpc ShowDDLLocks (ShowDDLLocksRequest) returns (ShowDDLLocksResponse) {} - // used by dmctl to manually unlock DDL lock - rpc UnlockDDLLock (UnlockDDLLockRequest) returns (UnlockDDLLockResponse) {} - - // OperateWorkerRelayTask requests some dm-workers to operate relay unit - rpc OperateWorkerRelayTask (OperateWorkerRelayRequest) returns (OperateWorkerRelayResponse) {} - - // PurgeWorkerRelay purges relay log files for some dm-workers - rpc PurgeWorkerRelay(PurgeWorkerRelayRequest) returns (PurgeWorkerRelayResponse) {} - - // CheckTask checks legality of task configuration - rpc CheckTask(CheckTaskRequest) returns (CheckTaskResponse) {} - - // Operate an upstream MySQL source. - rpc OperateSource(OperateSourceRequest) returns (OperateSourceResponse) { - option (google.api.http) = { - put: "/apis/v1alpha1/sources" - body: "*" - }; - } - - // RegisterWorker register the dm-workers. - rpc RegisterWorker(RegisterWorkerRequest) returns(RegisterWorkerResponse) {} - - // OfflineMember offline the dm cluster's members (master/worker). - rpc OfflineMember(OfflineMemberRequest) returns(OfflineMemberResponse) { - option (google.api.http) = { - delete: "/apis/v1alpha1/members/{type}/{name}" - }; - } - - // OperateLeader do some operate on master: - // - evict leader: make the master resign if it is leader, and will not campaign the leader again - // - cancel evict leader: the master can campaign leader again. - rpc OperateLeader(OperateLeaderRequest) returns(OperateLeaderResponse) { - option (google.api.http) = { - put: "/apis/v1alpha1/leader/{op}" - body: "*" - }; - } - - // ListMember list member information - rpc ListMember(ListMemberRequest) returns(ListMemberResponse) { - option (google.api.http) = { - get: "/apis/v1alpha1/members" - }; - } - - rpc OperateSchema(OperateSchemaRequest) returns(OperateSchemaResponse) { - option (google.api.http) = { - put: "/apis/v1alpha1/schema" - body: "*" - }; - } - - rpc GetSubTaskCfg(GetSubTaskCfgRequest) returns(GetSubTaskCfgResponse) { - option (google.api.http) = { - get: "/apis/v1alpha1/subtasks/{name}" - }; - } - - // GetCfg get config - rpc GetCfg(GetCfgRequest) returns(GetCfgResponse) { - option (google.api.http) = { - get: "/apis/v1alpha1/tasks/{name}" - }; - } - - rpc HandleError(HandleErrorRequest) returns(HandleErrorResponse) { - option (google.api.http) = { - put: "/apis/v1alpha1/errors" - body: "*" - }; - } - - rpc GetMasterCfg(GetMasterCfgRequest) returns(GetMasterCfgResponse) {} - - rpc TransferSource(TransferSourceRequest) returns(TransferSourceResponse) {} - - rpc OperateRelay(OperateRelayRequest) returns(OperateRelayResponse) {} + rpc StartTask (StartTaskRequest) returns (StartTaskResponse) { + option (google.api.http) = { + post: "/apis/v1alpha1/tasks" + body: "*" + }; + } + + rpc OperateTask (OperateTaskRequest) returns (OperateTaskResponse) { + option (google.api.http) = { + put: "/apis/v1alpha1/tasks/{name}" + body: "*" + }; + } + rpc UpdateTask (UpdateTaskRequest) returns (UpdateTaskResponse) {} + + rpc QueryStatus (QueryStatusListRequest) returns (QueryStatusListResponse) { + option (google.api.http) = { + get: "/apis/v1alpha1/status/{name}" + }; + } + + // show un-resolved DDL locks + rpc ShowDDLLocks (ShowDDLLocksRequest) returns (ShowDDLLocksResponse) {} + // used by dmctl to manually unlock DDL lock + rpc UnlockDDLLock (UnlockDDLLockRequest) returns (UnlockDDLLockResponse) {} + + // OperateWorkerRelayTask requests some dm-workers to operate relay unit + rpc OperateWorkerRelayTask (OperateWorkerRelayRequest) returns (OperateWorkerRelayResponse) {} + + // PurgeWorkerRelay purges relay log files for some dm-workers + rpc PurgeWorkerRelay(PurgeWorkerRelayRequest) returns (PurgeWorkerRelayResponse) {} + + // CheckTask checks legality of task configuration + rpc CheckTask(CheckTaskRequest) returns (CheckTaskResponse) {} + + // Operate an upstream MySQL source. + rpc OperateSource(OperateSourceRequest) returns (OperateSourceResponse) { + option (google.api.http) = { + put: "/apis/v1alpha1/sources" + body: "*" + }; + } + + // RegisterWorker register the dm-workers. + rpc RegisterWorker(RegisterWorkerRequest) returns(RegisterWorkerResponse) {} + + // OfflineMember offline the dm cluster's members (master/worker). + rpc OfflineMember(OfflineMemberRequest) returns(OfflineMemberResponse) { + option (google.api.http) = { + delete: "/apis/v1alpha1/members/{type}/{name}" + }; + } + + // OperateLeader do some operate on master: + // - evict leader: make the master resign if it is leader, and will not campaign the leader again + // - cancel evict leader: the master can campaign leader again. + rpc OperateLeader(OperateLeaderRequest) returns(OperateLeaderResponse) { + option (google.api.http) = { + put: "/apis/v1alpha1/leader/{op}" + body: "*" + }; + } + + // ListMember list member information + rpc ListMember(ListMemberRequest) returns(ListMemberResponse) { + option (google.api.http) = { + get: "/apis/v1alpha1/members" + }; + } + + rpc OperateSchema(OperateSchemaRequest) returns(OperateSchemaResponse) { + option (google.api.http) = { + put: "/apis/v1alpha1/schema" + body: "*" + }; + } + + rpc GetSubTaskCfg(GetSubTaskCfgRequest) returns(GetSubTaskCfgResponse) { + option (google.api.http) = { + get: "/apis/v1alpha1/subtasks/{name}" + }; + } + + // GetCfg get config + rpc GetCfg(GetCfgRequest) returns(GetCfgResponse) { + option (google.api.http) = { + get: "/apis/v1alpha1/tasks/{name}" + }; + } + + rpc HandleError(HandleErrorRequest) returns(HandleErrorResponse) { + option (google.api.http) = { + put: "/apis/v1alpha1/errors" + body: "*" + }; + } + + rpc GetMasterCfg(GetMasterCfgRequest) returns(GetMasterCfgResponse) {} + + rpc TransferSource(TransferSourceRequest) returns(TransferSourceResponse) {} + + rpc OperateRelay(OperateRelayRequest) returns(OperateRelayResponse) {} } message StartTaskRequest { - string task = 1; // task's configuration, yaml format - repeated string sources = 2; // mysql source need to do start task, empty for all sources defiend in the task config - bool removeMeta = 3; // whether to remove meta data for this task or not + string task = 1; // task's configuration, yaml format + repeated string sources = 2; // mysql source need to do start task, empty for all sources defined in the task config + bool removeMeta = 3; // whether to remove meta data for this task or not + string startTime = 4; // a highest priority field to specify starting of binlog replication } message StartTaskResponse { - bool result = 1; - string msg = 2; - repeated CommonWorkerResponse sources = 3; + bool result = 1; + string msg = 2; + repeated CommonWorkerResponse sources = 3; } message OperateTaskRequest { - TaskOp op = 1; // Stop / Pause / Resume - string name = 2; // task's name - repeated string sources = 3; // sources need to do operation, empty for matched sources in processing the task + TaskOp op = 1; // Stop / Pause / Resume + string name = 2; // task's name + repeated string sources = 3; // sources need to do operation, empty for matched sources in processing the task } message OperateTaskResponse { - TaskOp op = 1; - bool result = 2; - string msg = 3; - repeated CommonWorkerResponse sources = 4; + TaskOp op = 1; + bool result = 2; + string msg = 3; + repeated CommonWorkerResponse sources = 4; } @@ -142,26 +143,26 @@ message OperateTaskResponse { // support update partial config for syncer, loader, etc later // sources need to do update, empty for all sources in processing the task message UpdateTaskRequest { - string task = 1; - repeated string sources = 2; + string task = 1; + repeated string sources = 2; } message UpdateTaskResponse { - bool result = 1; - string msg = 2; - repeated CommonWorkerResponse sources = 3; + bool result = 1; + string msg = 2; + repeated CommonWorkerResponse sources = 3; } message QueryStatusListRequest { - string name = 1; // task's name, empty for all tasks - repeated string sources = 2; // sources need to query, empty for all sources + string name = 1; // task's name, empty for all tasks + repeated string sources = 2; // sources need to query, empty for all sources } message QueryStatusListResponse { - bool result = 1; - string msg = 2; - repeated QueryStatusResponse sources = 3; + bool result = 1; + string msg = 2; + repeated QueryStatusResponse sources = 3; } // ShowDDLLocksRequest used to query DDL locks which are un-resolved @@ -170,8 +171,8 @@ message QueryStatusListResponse { // any DDL lock in which the source is synced or unsynced will return // if specify task and sources both, and sources not doing the task , it will return empty DDL locks message ShowDDLLocksRequest { - string task = 1; - repeated string sources = 2; // sources need to query, empty for all sources + string task = 1; + repeated string sources = 2; // sources need to query, empty for all sources } // DDLLock represents a DDL lock info (I known the name confused with DDLLockInfo, any suggestion?) @@ -184,19 +185,19 @@ message ShowDDLLocksRequest { // synced: already synced dm-workers // unsynced: pending to sync dm-workers message DDLLock { - string ID = 1; - string task = 2; - string mode = 3; - string owner = 4; - repeated string DDLs = 5; - repeated string synced = 6; - repeated string unsynced = 7; + string ID = 1; + string task = 2; + string mode = 3; + string owner = 4; + repeated string DDLs = 5; + repeated string synced = 6; + repeated string unsynced = 7; } message ShowDDLLocksResponse { - bool result = 1; - string msg = 2; - repeated DDLLock locks = 3; // all un-resolved DDL locks + bool result = 1; + string msg = 2; + repeated DDLLock locks = 3; // all un-resolved DDL locks } // UnlockDDLLockRequest used to unlock (resolve) DDL lock manually @@ -204,27 +205,27 @@ message ShowDDLLocksResponse { // replaceOwner: dm-worker used to replace the original DDL lock's owner // forceRemove: force to remove the DDL lock even fail to execute the DDL for the owner. message UnlockDDLLockRequest { - string ID = 1; - string replaceOwner = 2; - bool forceRemove = 3; + string ID = 1; + string replaceOwner = 2; + bool forceRemove = 3; } message UnlockDDLLockResponse { - bool result = 1; - string msg = 2; + bool result = 1; + string msg = 2; } // OperateWorkerRelayRequest represents a request for some dm-workers to operate relay unit message OperateWorkerRelayRequest { - RelayOp op = 1; // Stop / Pause / Resume - repeated string sources = 2; + RelayOp op = 1; // Stop / Pause / Resume + repeated string sources = 2; } message OperateWorkerRelayResponse { - RelayOp op = 1; - bool result = 2; - string msg = 3; - repeated CommonWorkerResponse sources = 4; + RelayOp op = 1; + bool result = 2; + string msg = 3; + repeated CommonWorkerResponse sources = 4; } // PurgeWorkerRelayRequest represents a request to purge relay log files for some dm-workers @@ -234,232 +235,234 @@ message OperateWorkerRelayResponse { // filename: whether purge relay log files before this filename // subDir: specify relay sub directory for @filename message PurgeWorkerRelayRequest { - repeated string sources = 1; - bool inactive = 2; - int64 time = 3; - string filename = 4; - string subDir = 5; + repeated string sources = 1; + bool inactive = 2; + int64 time = 3; + string filename = 4; + string subDir = 5; } message PurgeWorkerRelayResponse { - bool result = 1; - string msg = 2; - repeated CommonWorkerResponse sources = 3; + bool result = 1; + string msg = 2; + repeated CommonWorkerResponse sources = 3; } message CheckTaskRequest { - string task = 1; // task's configuration, yaml format - int64 errCnt = 2; // max error count to display - int64 warnCnt = 3; // max warn count to display + string task = 1; // task's configuration, yaml format + int64 errCnt = 2; // max error count to display + int64 warnCnt = 3; // max warn count to display + string startTime = 4; // a highest priority field to specify starting of binlog replication } message CheckTaskResponse { - bool result = 1; - string msg = 2; + bool result = 1; + string msg = 2; } enum SourceOp { - InvalidSourceOp = 0; - StartSource = 1; - UpdateSource = 2; - StopSource = 3; - ShowSource = 4; + InvalidSourceOp = 0; + StartSource = 1; + UpdateSource = 2; + StopSource = 3; + ShowSource = 4; } message OperateSourceRequest { - SourceOp op = 1; - repeated string config = 2; - repeated string sourceID = 3; + SourceOp op = 1; + repeated string config = 2; + repeated string sourceID = 3; + string workerName = 4; } message OperateSourceResponse { - bool result = 1; - string msg = 2; - repeated CommonWorkerResponse sources = 3; + bool result = 1; + string msg = 2; + repeated CommonWorkerResponse sources = 3; } message RegisterWorkerRequest { - string name = 1; - string address = 2; + string name = 1; + string address = 2; } message RegisterWorkerResponse { - bool result = 1; - string msg = 2; + bool result = 1; + string msg = 2; } message OfflineMemberRequest { - string type = 1; - string name = 2; + string type = 1; + string name = 2; } message OfflineMemberResponse { - bool result = 1; - string msg = 2; + bool result = 1; + string msg = 2; } enum LeaderOp { - InvalidLeaderOp = 0; - EvictLeaderOp = 1; - CancelEvictLeaderOp = 2; + InvalidLeaderOp = 0; + EvictLeaderOp = 1; + CancelEvictLeaderOp = 2; } message OperateLeaderRequest { - LeaderOp op = 1; + LeaderOp op = 1; } message OperateLeaderResponse { - bool result = 1; - string msg = 2; + bool result = 1; + string msg = 2; } message MasterInfo { - string name = 1; - uint64 memberID = 2; - bool alive = 3; - repeated string peerURLs = 4; - repeated string clientURLs = 5; + string name = 1; + uint64 memberID = 2; + bool alive = 3; + repeated string peerURLs = 4; + repeated string clientURLs = 5; } message WorkerInfo { - string name = 1; - string addr = 2; - string stage = 3; - string source = 4; + string name = 1; + string addr = 2; + string stage = 3; + string source = 4; } message ListLeaderMember { - string msg = 1; - string name = 2; - string addr = 3; + string msg = 1; + string name = 2; + string addr = 3; } message ListMasterMember { - string msg = 1; - repeated MasterInfo masters = 2; + string msg = 1; + repeated MasterInfo masters = 2; } message ListWorkerMember { - string msg = 1; - repeated WorkerInfo workers = 2; + string msg = 1; + repeated WorkerInfo workers = 2; } message Members { - oneof member { - ListLeaderMember leader = 1; - ListMasterMember master = 2; - ListWorkerMember worker = 3; - } + oneof member { + ListLeaderMember leader = 1; + ListMasterMember master = 2; + ListWorkerMember worker = 3; + } } message ListMemberRequest { - bool leader = 1; - bool master = 2; - bool worker = 3; - repeated string names = 4; + bool leader = 1; + bool master = 2; + bool worker = 3; + repeated string names = 4; } message ListMemberResponse { - bool result = 1; - string msg = 2; - repeated Members members = 3; + bool result = 1; + string msg = 2; + repeated Members members = 3; } message OperateSchemaRequest { - SchemaOp op = 1; // operation type - string task = 2; // task name - repeated string sources = 3; // source ID list - string database = 4; // database name - string table = 5; // table name - string schema = 6; // schema content, a `CREATE TABLE` statement - bool flush = 7; // flush table info and checkpoint - bool sync = 8; // sync the table info to master - bool fromSource = 9; // update schema from source schema - bool fromTarget = 10; // update schema from target schema + SchemaOp op = 1; // operation type + string task = 2; // task name + repeated string sources = 3; // source ID list + string database = 4; // database name + string table = 5; // table name + string schema = 6; // schema content, a `CREATE TABLE` statement + bool flush = 7; // flush table info and checkpoint + bool sync = 8; // sync the table info to master + bool fromSource = 9; // update schema from source schema + bool fromTarget = 10; // update schema from target schema } message OperateSchemaResponse { - bool result = 1; - string msg = 2; - repeated CommonWorkerResponse sources = 3; + bool result = 1; + string msg = 2; + repeated CommonWorkerResponse sources = 3; } message GetSubTaskCfgRequest { - // the task name - string name = 1; + // the task name + string name = 1; } message GetSubTaskCfgResponse { - bool result = 1; - string msg = 2; - repeated string cfgs = 3; + bool result = 1; + string msg = 2; + repeated string cfgs = 3; } enum CfgType { - InvalidType = 0; - TaskType = 1; - MasterType = 2; - WorkerType = 3; - SourceType = 4; - TaskTemplateType = 5; + InvalidType = 0; + TaskType = 1; + MasterType = 2; + WorkerType = 3; + SourceType = 4; + TaskTemplateType = 5; } message GetCfgRequest { - CfgType type = 1; // the config type - string name = 2; // the config name + CfgType type = 1; // the config type + string name = 2; // the config name } message GetCfgResponse { - bool result = 1; - string msg = 2; - string cfg = 3; + bool result = 1; + string msg = 2; + string cfg = 3; } message GetMasterCfgRequest { } message GetMasterCfgResponse { - string cfg = 1; + string cfg = 1; } message HandleErrorRequest { - ErrorOp op = 1; // operation type - string task = 2; // the task name - repeated string sources = 3; // source ID list - string binlogPos = 4; // binlog-pos (that's file:pos format) - repeated string sqls = 5; // sqls (use for replace) + ErrorOp op = 1; // operation type + string task = 2; // the task name + repeated string sources = 3; // source ID list + string binlogPos = 4; // binlog-pos (that's file:pos format) + repeated string sqls = 5; // sqls (use for replace) } message HandleErrorResponse { - bool result = 1; - string msg = 2; - repeated CommonWorkerResponse sources = 3; + bool result = 1; + string msg = 2; + repeated CommonWorkerResponse sources = 3; } message TransferSourceRequest { - string source = 1; - string worker = 2; + string source = 1; + string worker = 2; } message TransferSourceResponse { - bool result = 1; - string msg = 2; + bool result = 1; + string msg = 2; } message OperateRelayRequest { - RelayOpV2 op = 1; - string source = 2; - repeated string worker = 3; + RelayOpV2 op = 1; + string source = 2; + repeated string worker = 3; } message OperateRelayResponse { - bool result = 1; - string msg = 2; - repeated CommonWorkerResponse sources = 3; + bool result = 1; + string msg = 2; + repeated CommonWorkerResponse sources = 3; } enum RelayOpV2 { - InvalidRelayOpV2 = 0; - StartRelayV2 = 1; - StopRelayV2 = 2; + InvalidRelayOpV2 = 0; + StartRelayV2 = 1; + StopRelayV2 = 2; } diff --git a/dm/dm/worker/subtask.go b/dm/dm/worker/subtask.go index 8615e76010b..ca04012059a 100644 --- a/dm/dm/worker/subtask.go +++ b/dm/dm/worker/subtask.go @@ -61,20 +61,12 @@ func createRealUnits(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, wor switch cfg.Mode { case config.ModeAll: us = append(us, dumpling.NewDumpling(cfg)) - if cfg.NeedUseLightning() { - us = append(us, loader.NewLightning(cfg, etcdClient, workerName)) - } else { - us = append(us, loader.NewLoader(cfg, etcdClient, workerName)) - } + us = append(us, newLoadUnit(cfg, etcdClient, workerName)) us = append(us, syncer.NewSyncer(cfg, etcdClient, relay)) case config.ModeFull: // NOTE: maybe need another checker in the future? us = append(us, dumpling.NewDumpling(cfg)) - if cfg.NeedUseLightning() { - us = append(us, loader.NewLightning(cfg, etcdClient, workerName)) - } else { - us = append(us, loader.NewLoader(cfg, etcdClient, workerName)) - } + us = append(us, newLoadUnit(cfg, etcdClient, workerName)) case config.ModeIncrement: us = append(us, syncer.NewSyncer(cfg, etcdClient, relay)) default: @@ -83,6 +75,21 @@ func createRealUnits(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, wor return us } +func newLoadUnit(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, workerName string) unit.Unit { + hasAutoGenColumn := false + for _, rule := range cfg.RouteRules { + if rule.SchemaExtractor != nil || rule.TableExtractor != nil || rule.SourceExtractor != nil { + hasAutoGenColumn = true + break + } + } + // tidb-lightning doesn't support column mapping currently + if cfg.ImportMode == config.LoadModeLoader || cfg.OnDuplicate == config.OnDuplicateError || hasAutoGenColumn || len(cfg.ColumnMappingRules) > 0 { + return loader.NewLoader(cfg, etcdClient, workerName) + } + return loader.NewLightning(cfg, etcdClient, workerName) +} + // SubTask represents a sub task of data migration. type SubTask struct { cfg *config.SubTaskConfig diff --git a/dm/dm/worker/subtask_test.go b/dm/dm/worker/subtask_test.go index 8fb5006f03a..41e7a5a2152 100644 --- a/dm/dm/worker/subtask_test.go +++ b/dm/dm/worker/subtask_test.go @@ -55,7 +55,7 @@ func (t *testSubTask) TestCreateUnits(c *C) { c.Assert(unitsFull, HasLen, 2) _, ok := unitsFull[0].(*dumpling.Dumpling) c.Assert(ok, IsTrue) - _, ok = unitsFull[1].(*loader.Loader) + _, ok = unitsFull[1].(*loader.LightningLoader) c.Assert(ok, IsTrue) cfg.Mode = config.ModeIncrement @@ -69,7 +69,7 @@ func (t *testSubTask) TestCreateUnits(c *C) { c.Assert(unitsAll, HasLen, 3) _, ok = unitsAll[0].(*dumpling.Dumpling) c.Assert(ok, IsTrue) - _, ok = unitsAll[1].(*loader.Loader) + _, ok = unitsAll[1].(*loader.LightningLoader) c.Assert(ok, IsTrue) _, ok = unitsAll[2].(*syncer.Syncer) c.Assert(ok, IsTrue) diff --git a/dm/errors.toml b/dm/errors.toml index 22f1bbe6bc7..96bab495cf8 100644 --- a/dm/errors.toml +++ b/dm/errors.toml @@ -1102,6 +1102,18 @@ description = "" workaround = "Please check the `collation_compatible` config in task configuration file, which can be set to `loose`/`strict`." tags = ["internal", "medium"] +[error.DM-config-20053] +message = "invalid load mode '%s'" +description = "" +workaround = "Please choose a valid value in ['sql', 'loader']" +tags = ["internal", "medium"] + +[error.DM-config-20054] +message = "invalid load on-duplicate '%s'" +description = "" +workaround = "Please choose a valid value in ['replace', 'error', 'ignore']" +tags = ["internal", "medium"] + [error.DM-binlog-op-22001] message = "" description = "" @@ -1624,12 +1636,6 @@ description = "" workaround = "If you want to redo the whole task, please check that you have not forgotten to add -remove-meta flag for start-task command." tags = ["internal", "high"] -[error.DM-functional-34019] -message = "DM do not support backend %s " -description = "" -workaround = "If you do not understand the configure `tidb.backend` you can just delete it." -tags = ["internal", "high"] - [error.DM-sync-unit-36001] message = "panic error: %v" description = "" @@ -2374,6 +2380,12 @@ description = "" workaround = "" tags = ["internal", "high"] +[error.DM-dm-master-38056] +message = "downstream database config and meta for task %s not found" +description = "" +workaround = "" +tags = ["internal", "high"] + [error.DM-dm-worker-40001] message = "parse dm-worker config flag set" description = "" @@ -3214,6 +3226,12 @@ description = "" workaround = "Please pause task by `dmctl pause-task`." tags = ["internal", "low"] +[error.DM-scheduler-46033] +message = "dm-worker with name %s not free" +description = "" +workaround = "" +tags = ["internal", "low"] + [error.DM-dmctl-48001] message = "can not create grpc connection" description = "" diff --git a/dm/loader/checkpoint.go b/dm/loader/checkpoint.go index c99afa2a34c..30575d258fc 100644 --- a/dm/loader/checkpoint.go +++ b/dm/loader/checkpoint.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tiflow/dm/pkg/conn" tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/cputil" + fr "github.com/pingcap/tiflow/dm/pkg/func-rollback" "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/terror" @@ -101,7 +102,18 @@ type RemoteCheckPoint struct { } func newRemoteCheckPoint(tctx *tcontext.Context, cfg *config.SubTaskConfig, id string) (CheckPoint, error) { - db, dbConns, err := createConns(tctx, cfg, cfg.Name, cfg.SourceID, 1) + var err error + var db *conn.BaseDB + var dbConns []*DBConn + + rollbackHolder := fr.NewRollbackHolder("loader") + defer func() { + if err != nil { + rollbackHolder.RollbackReverseOrder() + } + }() + + db, dbConns, err = createConns(tctx, cfg, cfg.Name, cfg.SourceID, 1) if err != nil { return nil, err } @@ -116,6 +128,7 @@ func newRemoteCheckPoint(tctx *tcontext.Context, cfg *config.SubTaskConfig, id s logger: tctx.L().WithFields(zap.String("component", "remote checkpoint")), } cp.restoringFiles.pos = make(map[string]map[string]FilePosSet) + rollbackHolder.Add(fr.FuncRollback{Name: "CloseRemoteCheckPoint", Fn: cp.Close}) err = cp.prepare(tctx) if err != nil { diff --git a/dm/loader/lightning.go b/dm/loader/lightning.go index 66b41fdf347..08cc1bc5df1 100644 --- a/dm/loader/lightning.go +++ b/dm/loader/lightning.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/br/pkg/lightning" + "github.com/pingcap/tidb/br/pkg/lightning/checkpoints" lcfg "github.com/pingcap/tidb/br/pkg/lightning/config" "go.etcd.io/etcd/clientv3" "go.uber.org/atomic" @@ -48,6 +49,7 @@ type LightningLoader struct { sync.RWMutex timeZone string + sqlMode string lightningGlobalConfig *lcfg.GlobalConfig cfg *config.SubTaskConfig @@ -95,12 +97,9 @@ func makeGlobalConfig(cfg *config.SubTaskConfig) *lcfg.GlobalConfig { lightningCfg.TiDB.Psw = cfg.To.Password lightningCfg.TiDB.User = cfg.To.User lightningCfg.TiDB.Port = cfg.To.Port - lightningCfg.TiDB.StatusPort = cfg.TiDB.StatusPort - lightningCfg.TiDB.PdAddr = cfg.TiDB.PdAddr - lightningCfg.TiDB.LogLevel = cfg.LogLevel - lightningCfg.TikvImporter.Backend = cfg.TiDB.Backend + lightningCfg.TikvImporter.Backend = lcfg.BackendTiDB lightningCfg.PostRestore.Checksum = lcfg.OpLevelOff - if cfg.TiDB.Backend == lcfg.BackendLocal { + if lightningCfg.TikvImporter.Backend == lcfg.BackendLocal { lightningCfg.TikvImporter.SortedKVDir = cfg.Dir } lightningCfg.Mydumper.SourceDir = cfg.Dir @@ -148,25 +147,64 @@ func (l *LightningLoader) Init(ctx context.Context) (err error) { } } l.timeZone = timeZone + + for k, v := range l.cfg.To.Session { + if strings.ToLower(k) == "sql_mode" { + l.sqlMode = v + break + } + } + + if len(l.sqlMode) == 0 { + sqlModes, err3 := utils.AdjustSQLModeCompatible(l.cfg.LoaderConfig.SQLMode) + if err3 != nil { + l.logger.Warn("cannot adjust sql_mode compatible, the sql_mode will stay the same", log.ShortError(err3)) + } + l.sqlMode = sqlModes + } return nil } +func (l *LightningLoader) ignoreCheckpointError(ctx context.Context, cfg *lcfg.Config) error { + status, err := l.checkPointList.taskStatus(ctx) + if err != nil { + return err + } + if status != lightningStatusRunning { + return nil + } + cpdb, err := checkpoints.OpenCheckpointsDB(ctx, cfg) + if err != nil { + return err + } + defer func() { + _ = cpdb.Close() + }() + return errors.Trace(cpdb.IgnoreErrorCheckpoint(ctx, "all")) +} + func (l *LightningLoader) runLightning(ctx context.Context, cfg *lcfg.Config) error { taskCtx, cancel := context.WithCancel(ctx) l.Lock() l.cancel = cancel l.Unlock() - if err := l.checkPointList.UpdateStatus(ctx, lightningStatusRunning); err != nil { + + // always try to skill all checkpoint errors so we can resume this phase. + err := l.ignoreCheckpointError(ctx, cfg) + if err != nil { + l.logger.Warn("check lightning checkpoint status failed, skip this error", log.ShortError(err)) + } + if err = l.checkPointList.UpdateStatus(ctx, lightningStatusRunning); err != nil { return err } - err := l.core.RunOnce(taskCtx, cfg, nil) - failpoint.Inject("LightningLoadDataSlowDown", nil) - failpoint.Inject("LightningLoadDataSlowDownByTask", func(val failpoint.Value) { + err = l.core.RunOnce(taskCtx, cfg, nil) + failpoint.Inject("LoadDataSlowDown", nil) + failpoint.Inject("LoadDataSlowDownByTask", func(val failpoint.Value) { tasks := val.(string) taskNames := strings.Split(tasks, ",") for _, taskName := range taskNames { if l.cfg.Name == taskName { - l.logger.Info("inject failpoint LightningLoadDataSlowDownByTask", zap.String("task", taskName)) + l.logger.Info("inject failpoint LoadDataSlowDownByTask in lightning loader", zap.String("task", taskName)) <-taskCtx.Done() } } @@ -197,14 +235,20 @@ func (l *LightningLoader) restore(ctx context.Context) error { cpPath := filepath.Join(l.cfg.LoaderConfig.Dir, lightningCheckpointFileName) cfg.Checkpoint.DSN = cpPath cfg.Checkpoint.KeepAfterSuccess = lcfg.CheckpointOrigin + cfg.TikvImporter.OnDuplicate = string(l.cfg.OnDuplicate) cfg.TiDB.Vars = make(map[string]string) + cfg.Routes = l.cfg.RouteRules if l.cfg.To.Session != nil { for k, v := range l.cfg.To.Session { cfg.TiDB.Vars[k] = v } } - cfg.TiDB.StrSQLMode = l.cfg.LoaderConfig.SQLMode - cfg.TiDB.Vars = map[string]string{"time_zone": l.timeZone} + cfg.TiDB.StrSQLMode = l.sqlMode + cfg.TiDB.Vars = map[string]string{ + "time_zone": l.timeZone, + // always set transaction mode to optimistic + "tidb_txn_mode": "optimistic", + } err = l.runLightning(ctx, cfg) if err == nil { l.finish.Store(true) @@ -264,7 +308,12 @@ func (l *LightningLoader) Process(ctx context.Context, pr chan pb.ProcessResult) isCanceled = true default: } - l.logger.Info("lightning load end", zap.Bool("IsCanceled", isCanceled)) + s := l.status() + l.logger.Info("lightning load end", + zap.Bool("IsCanceled", isCanceled), + zap.Int64("finished_bytes", s.FinishedBytes), + zap.Int64("total_bytes", s.TotalBytes), + zap.String("progress", s.Progress)) pr <- pb.ProcessResult{IsCanceled: isCanceled, Errors: errs} } @@ -324,8 +373,7 @@ func (l *LightningLoader) Update(ctx context.Context, cfg *config.SubTaskConfig) return nil } -// Status returns the unit's current status. -func (l *LightningLoader) Status(_ *binlog.SourceStatus) interface{} { +func (l *LightningLoader) status() *pb.LoadStatus { finished, total := l.core.Status() progress := percent(finished, total, l.finish.Load()) s := &pb.LoadStatus{ @@ -337,3 +385,8 @@ func (l *LightningLoader) Status(_ *binlog.SourceStatus) interface{} { } return s } + +// Status returns the unit's current status. +func (l *LightningLoader) Status(_ *binlog.SourceStatus) interface{} { + return l.status() +} diff --git a/dm/pkg/schema/tracker.go b/dm/pkg/schema/tracker.go index 3c15c3eb05f..1d0640cc6f9 100644 --- a/dm/pkg/schema/tracker.go +++ b/dm/pkg/schema/tracker.go @@ -345,7 +345,7 @@ func (tr *Tracker) CreateSchemaIfNotExists(db string) error { if tr.dom.InfoSchema().SchemaExists(dbName) { return nil } - return tr.dom.DDL().CreateSchema(tr.se, dbName, nil, nil, nil) + return tr.dom.DDL().CreateSchema(tr.se, dbName, nil, nil) } // cloneTableInfo creates a clone of the TableInfo. @@ -370,6 +370,22 @@ func (tr *Tracker) CreateTableIfNotExists(table *filter.Table, ti *model.TableIn return tr.dom.DDL().CreateTableWithInfo(tr.se, schemaName, ti, ddl.OnExistIgnore) } +func (tr *Tracker) BatchCreateTableIfNotExist(tablesToCreate map[string]map[string]*model.TableInfo) error { + for schema, tableNameInfo := range tablesToCreate { + var cloneTis []*model.TableInfo + for table, ti := range tableNameInfo { + cloneTi := cloneTableInfo(ti) // clone TableInfo w.r.t the warning of the CreateTable function + cloneTi.Name = model.NewCIStr(table) // TableInfo has no `TableName` + cloneTis = append(cloneTis, cloneTi) + } + schemaName := model.NewCIStr(schema) + if err := tr.dom.DDL().BatchCreateTableWithInfo(tr.se, schemaName, cloneTis, ddl.OnExistIgnore); err != nil { + return err + } + } + return nil +} + // GetSystemVar gets a variable from schema tracker. func (tr *Tracker) GetSystemVar(name string) (string, bool) { return tr.se.GetSessionVars().GetSystemVar(name) diff --git a/dm/pkg/schema/tracker_test.go b/dm/pkg/schema/tracker_test.go index 5e6d9f12e97..b67ac50483c 100644 --- a/dm/pkg/schema/tracker_test.go +++ b/dm/pkg/schema/tracker_test.go @@ -445,6 +445,118 @@ func (s *trackerSuite) TestCreateTableIfNotExists(c *C) { c.Assert(duration.Seconds(), Less, float64(30)) } +func (s *trackerSuite) TestBatchCreateTableIfNotExist(c *C) { + log.SetLevel(zapcore.ErrorLevel) + tracker, err := NewTracker(context.Background(), "test-tracker", defaultTestSessionCfg, s.dbConn) + c.Assert(err, IsNil) + err = tracker.CreateSchemaIfNotExists("testdb") + c.Assert(err, IsNil) + err = tracker.CreateSchemaIfNotExists("testdb2") + c.Assert(err, IsNil) + + tables := []*filter.Table{ + { + Schema: "testdb", + Name: "foo", + }, + { + Schema: "testdb", + Name: "foo1", + }, + { + Schema: "testdb2", + Name: "foo3", + }, + } + execStmt := []string{ + `create table foo( + a int primary key auto_increment, + b int as (c+1) not null, + c int comment 'some cmt', + d text, + key dk(d(255)) + ) comment 'more cmt' partition by range columns (a) ( + partition x41 values less than (41), + partition x82 values less than (82), + partition rest values less than maxvalue comment 'part cmt' + );`, + `create table foo1( + a int primary key, + b text not null, + d datetime, + e varchar(5) + );`, + `create table foo3( + a int, + b int, + primary key(a));`, + } + tiInfos := make([]*model.TableInfo, len(tables)) + for i := range tables { + ctx := context.Background() + err = tracker.Exec(ctx, tables[i].Schema, execStmt[i]) + c.Assert(err, IsNil) + tiInfos[i], err = tracker.GetTableInfo(tables[i]) + c.Assert(err, IsNil) + c.Assert(tiInfos[i], NotNil) + c.Assert(tiInfos[i].Name.O, Equals, tables[i].Name) + tiInfos[i] = tiInfos[i].Clone() + clearVolatileInfo(tiInfos[i]) + } + // drop all tables and recover + // 1. drop + for i := range tables { + err = tracker.DropTable(tables[i]) + c.Assert(err, IsNil) + _, err = tracker.GetTableInfo(tables[i]) + c.Assert(err, ErrorMatches, `.*Table 'testdb.*\.foo.*' doesn't exist`) // drop table success + } + // 2. recover + tablesToCreate := map[string]map[string]*model.TableInfo{} + tablesToCreate["testdb"] = map[string]*model.TableInfo{} + tablesToCreate["testdb2"] = map[string]*model.TableInfo{} + for i := range tables { + tablesToCreate[tables[i].Schema][tables[i].Name] = tiInfos[i] + } + err = tracker.BatchCreateTableIfNotExist(tablesToCreate) + c.Assert(err, IsNil) + // 3. check all create success + for i := range tables { + var ti *model.TableInfo + ti, err = tracker.GetTableInfo(tables[i]) + c.Assert(err, IsNil) + cloneTi := ti.Clone() + clearVolatileInfo(cloneTi) + c.Assert(cloneTi, DeepEquals, tiInfos[i]) + } + + // drop two tables and create all three + // expect: silently succeed + // 1. drop table + err = tracker.DropTable(tables[2]) + c.Assert(err, IsNil) + err = tracker.DropTable(tables[0]) + c.Assert(err, IsNil) + // 2. batch create + err = tracker.BatchCreateTableIfNotExist(tablesToCreate) + c.Assert(err, IsNil) + // 3. check + for i := range tables { + var ti *model.TableInfo + ti, err = tracker.GetTableInfo(tables[i]) + c.Assert(err, IsNil) + clearVolatileInfo(ti) + c.Assert(ti, DeepEquals, tiInfos[i]) + } + + // drop schema and raise error + ctx := context.Background() + err = tracker.Exec(ctx, "", `drop database testdb`) + c.Assert(err, IsNil) + err = tracker.BatchCreateTableIfNotExist(tablesToCreate) + c.Assert(err, NotNil) +} + func (s *trackerSuite) TestAllSchemas(c *C) { log.SetLevel(zapcore.ErrorLevel) ctx := context.Background() diff --git a/dm/pkg/shardddl/optimism/info.go b/dm/pkg/shardddl/optimism/info.go index eb0179a7a58..bdb43c067ce 100644 --- a/dm/pkg/shardddl/optimism/info.go +++ b/dm/pkg/shardddl/optimism/info.go @@ -298,15 +298,14 @@ func deleteInfoOp(info Info) clientv3.Op { info.Task, info.Source, info.UpSchema, info.UpTable)) } -// ClearTestInfoOperationSchema is used to clear all shard DDL information in optimism mode. +// ClearTestInfoOperationColumns is used to clear all shard DDL information in optimism mode. // it only used for testing now. -func ClearTestInfoOperationSchema(cli *clientv3.Client) error { +func ClearTestInfoOperationColumn(cli *clientv3.Client) error { clearSource := clientv3.OpDelete(common.ShardDDLOptimismSourceTablesKeyAdapter.Path(), clientv3.WithPrefix()) clearInfo := clientv3.OpDelete(common.ShardDDLOptimismInfoKeyAdapter.Path(), clientv3.WithPrefix()) clearOp := clientv3.OpDelete(common.ShardDDLOptimismOperationKeyAdapter.Path(), clientv3.WithPrefix()) - clearISOp := clientv3.OpDelete(common.ShardDDLOptimismInitSchemaKeyAdapter.Path(), clientv3.WithPrefix()) clearColumns := clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Path(), clientv3.WithPrefix()) - _, err := cli.Txn(context.Background()).Then(clearSource, clearInfo, clearOp, clearISOp, clearColumns).Commit() + _, err := cli.Txn(context.Background()).Then(clearSource, clearInfo, clearOp, clearColumns).Commit() return err } diff --git a/dm/pkg/shardddl/optimism/info_test.go b/dm/pkg/shardddl/optimism/info_test.go index 63df2a422a1..44a35c1c41d 100644 --- a/dm/pkg/shardddl/optimism/info_test.go +++ b/dm/pkg/shardddl/optimism/info_test.go @@ -47,7 +47,7 @@ func TestInfo(t *testing.T) { // clear keys in etcd test cluster. func clearTestInfoOperation(c *C) { - c.Assert(ClearTestInfoOperationSchema(etcdTestCli), IsNil) + c.Assert(ClearTestInfoOperationColumn(etcdTestCli), IsNil) } func createTableInfo(c *C, p *parser.Parser, se sessionctx.Context, tableID int64, sql string) *model.TableInfo { diff --git a/dm/pkg/shardddl/optimism/keeper.go b/dm/pkg/shardddl/optimism/keeper.go index b88493e78f1..f1f0d3dc57c 100644 --- a/dm/pkg/shardddl/optimism/keeper.go +++ b/dm/pkg/shardddl/optimism/keeper.go @@ -20,79 +20,63 @@ import ( "github.com/pingcap/tidb-tools/pkg/schemacmp" "go.etcd.io/etcd/clientv3" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/terror" "github.com/pingcap/tiflow/dm/pkg/utils" ) +// DownstreamMeta used to fetch table info from downstream. +type DownstreamMeta struct { + dbConfig *config.DBConfig + meta string +} + // LockKeeper used to keep and handle DDL lock conveniently. // The lock information do not need to be persistent, and can be re-constructed from the shard DDL info. +// But the drop columns should be persistent. type LockKeeper struct { mu sync.RWMutex locks map[string]*Lock // lockID -> Lock + + downstreamMetaMap map[string]*DownstreamMeta + getDownstreamMetaFunc func(string) (*config.DBConfig, string) + // lockID -> column name -> source -> upSchema -> upTable -> int + dropColumns map[string]map[string]map[string]map[string]map[string]DropColumnStage } // NewLockKeeper creates a new LockKeeper instance. -func NewLockKeeper() *LockKeeper { +func NewLockKeeper(getDownstreamMetaFunc func(string) (*config.DBConfig, string)) *LockKeeper { return &LockKeeper{ - locks: make(map[string]*Lock), + locks: make(map[string]*Lock), + downstreamMetaMap: make(map[string]*DownstreamMeta), + getDownstreamMetaFunc: getDownstreamMetaFunc, } } -// RebuildLocksAndTables rebuild the locks and tables. -func (lk *LockKeeper) RebuildLocksAndTables( - cli *clientv3.Client, - ifm map[string]map[string]map[string]map[string]Info, - colm map[string]map[string]map[string]map[string]map[string]DropColumnStage, - lockJoined map[string]schemacmp.Table, - lockTTS map[string][]TargetTable, - missTable map[string]map[string]map[string]map[string]schemacmp.Table, -) { - var ( - lock *Lock - ok bool - ) - for task, taskInfos := range ifm { - for source, sourceInfos := range taskInfos { - for schema, schemaInfos := range sourceInfos { - for table, info := range schemaInfos { - lockID := utils.GenDDLLockID(info.Task, info.DownSchema, info.DownTable) - if lock, ok = lk.locks[lockID]; !ok { - lock = NewLock(cli, lockID, info.Task, info.DownSchema, info.DownTable, lockJoined[lockID], lockTTS[lockID]) - } - // filter info which doesn't have SourceTable - // SourceTable will be changed after user update block-allow-list - // But old infos still remain in etcd. - // TODO: add a mechanism to remove all outdated infos in etcd. - if !lock.TableExist(info.Source, info.UpSchema, info.UpTable) { - delete(ifm[task][source][schema], table) - continue - } - lk.locks[lockID] = lock - lock.tables[info.Source][info.UpSchema][info.UpTable] = schemacmp.Encode(info.TableInfoBefore) - if columns, ok := colm[lockID]; ok { - lock.columns = columns - } - } - } - } +// SetDropColumns set drop columns for lock keeper. +func (lk *LockKeeper) SetDropColumns(dropColumns map[string]map[string]map[string]map[string]map[string]DropColumnStage) { + lk.dropColumns = dropColumns +} + +// getDownstreamMeta gets and cached downstream meta. +func (lk *LockKeeper) getDownstreamMeta(task string) (*DownstreamMeta, error) { + if downstreamMeta, ok := lk.downstreamMetaMap[task]; ok { + return downstreamMeta, nil } - // update missTable's table info for locks - for lockID, lockTable := range missTable { - for source, sourceTable := range lockTable { - for schema, schemaTable := range sourceTable { - for table, tableinfo := range schemaTable { - if _, ok := lk.locks[lockID]; !ok { - continue - } - if !lk.locks[lockID].TableExist(source, schema, table) { - continue - } - lk.locks[lockID].tables[source][schema][table] = tableinfo - } - } - } + dbConfig, meta := lk.getDownstreamMetaFunc(task) + if dbConfig == nil { + return nil, terror.ErrMasterOptimisticDownstreamMetaNotFound.Generate(task) } + downstreamMeta := &DownstreamMeta{dbConfig: dbConfig, meta: meta} + lk.downstreamMetaMap[task] = downstreamMeta + return downstreamMeta, nil +} + +// RemoveDownstreamMeta removes downstream mate by task. +func (lk *LockKeeper) RemoveDownstreamMeta(task string) { + delete(lk.downstreamMetaMap, task) } // TrySync tries to sync the lock. @@ -111,8 +95,20 @@ func (lk *LockKeeper) TrySync(cli *clientv3.Client, info Info, tts []TargetTable } if l, ok = lk.locks[lockID]; !ok { - lk.locks[lockID] = NewLock(cli, lockID, info.Task, info.DownSchema, info.DownTable, schemacmp.Encode(info.TableInfoBefore), tts) + downstreamMeta, err := lk.getDownstreamMeta(info.Task) + if err != nil { + log.L().Error("get downstream meta", log.ShortError(err)) + } + + lk.locks[lockID] = NewLock(cli, lockID, info.Task, info.DownSchema, info.DownTable, schemacmp.Encode(info.TableInfoBefore), tts, downstreamMeta) l = lk.locks[lockID] + + // set drop columns, only when recover locks + if lk.dropColumns != nil { + if cols, ok := lk.dropColumns[lockID]; ok { + l.columns = cols + } + } } newDDLs, cols, err := l.TrySync(info, tts) @@ -143,6 +139,21 @@ func (lk *LockKeeper) FindLock(lockID string) *Lock { return lk.locks[lockID] } +// FindLocksByTask finds locks by task. +func (lk *LockKeeper) FindLocksByTask(task string) []*Lock { + lk.mu.RLock() + defer lk.mu.RUnlock() + + locks := make([]*Lock, 0) + for _, lock := range lk.locks { + if lock.Task == task { + locks = append(locks, lock) + } + } + + return locks +} + // FindLockByInfo finds a lock with a shard DDL info. func (lk *LockKeeper) FindLockByInfo(info Info) *Lock { return lk.FindLock(genDDLLockID(info)) @@ -166,6 +177,7 @@ func (lk *LockKeeper) Clear() { defer lk.mu.Unlock() lk.locks = make(map[string]*Lock) + lk.downstreamMetaMap = make(map[string]*DownstreamMeta) } // genDDLLockID generates DDL lock ID from its info. @@ -245,6 +257,30 @@ func (tk *TableKeeper) AddTable(task, source, upSchema, upTable, downSchema, dow return added } +// SourceTableExist check whether a source table exist. +func (tk *TableKeeper) SourceTableExist(task, source, upSchema, upTable, downSchema, downTable string) bool { + tk.mu.Lock() + defer tk.mu.Unlock() + + if _, ok := tk.tables[task]; !ok { + return false + } + if _, ok := tk.tables[task][source]; !ok { + return false + } + st := tk.tables[task][source] + targetTable := st.TargetTable(downSchema, downTable) + + if targetTable.UpTables != nil { + if tables, ok := targetTable.UpTables[upSchema]; ok { + if _, ok2 := tables[upTable]; ok2 { + return true + } + } + } + return false +} + // RemoveTable removes a table from the source tables. // it returns whether removed (exit before). func (tk *TableKeeper) RemoveTable(task, source, upSchema, upTable, downSchema, downTable string) bool { @@ -276,6 +312,20 @@ func (tk *TableKeeper) RemoveTableByTask(task string) bool { return true } +// RemoveTableByTaskAndSource removes tables from the source tables through task name and sources. +func (tk *TableKeeper) RemoveTableByTaskAndSources(task string, sources []string) { + tk.mu.Lock() + defer tk.mu.Unlock() + + if _, ok := tk.tables[task]; !ok { + return + } + + for _, source := range sources { + delete(tk.tables[task], source) + } +} + // FindTables finds source tables by task name and downstream table name. func (tk *TableKeeper) FindTables(task, downSchema, downTable string) []TargetTable { tk.mu.RLock() diff --git a/dm/pkg/shardddl/optimism/keeper_test.go b/dm/pkg/shardddl/optimism/keeper_test.go index 6b686872c66..5aefe4173c2 100644 --- a/dm/pkg/shardddl/optimism/keeper_test.go +++ b/dm/pkg/shardddl/optimism/keeper_test.go @@ -17,13 +17,14 @@ import ( "testing" . "github.com/pingcap/check" - "github.com/pingcap/tidb-tools/pkg/schemacmp" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/util/mock" "go.etcd.io/etcd/integration" - "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/pkg/terror" ) type testKeeper struct{} @@ -41,7 +42,7 @@ func TestKeeper(t *testing.T) { func (t *testKeeper) TestLockKeeper(c *C) { var ( - lk = NewLockKeeper() + lk = NewLockKeeper(getDownstreamMeta) upSchema = "foo_1" upTable = "bar_1" downSchema = "foo" @@ -81,6 +82,13 @@ func (t *testKeeper) TestLockKeeper(c *C) { c.Assert(lock1, NotNil) c.Assert(lock1.ID, Equals, lockID1) c.Assert(lk.FindLockByInfo(i11).ID, Equals, lockID1) + + lks := lk.FindLocksByTask("hahaha") + c.Assert(len(lks), Equals, 0) + lks = lk.FindLocksByTask(task1) + c.Assert(len(lks), Equals, 1) + c.Assert(lks[0].ID, Equals, lockID1) + synced, remain := lock1.IsSynced() c.Assert(synced, IsFalse) c.Assert(remain, Equals, 1) @@ -110,6 +118,13 @@ func (t *testKeeper) TestLockKeeper(c *C) { c.Assert(synced, IsTrue) c.Assert(remain, Equals, 0) + lks = lk.FindLocksByTask(task1) + c.Assert(len(lks), Equals, 1) + c.Assert(lks[0].ID, Equals, lockID1) + lks = lk.FindLocksByTask(task2) + c.Assert(len(lks), Equals, 1) + c.Assert(lks[0].ID, Equals, lockID2) + // try to find not-exists lock. lockIDNotExists := "lock-not-exists" c.Assert(lk.FindLock(lockIDNotExists), IsNil) @@ -134,7 +149,7 @@ func (t *testKeeper) TestLockKeeper(c *C) { func (t *testKeeper) TestLockKeeperMultipleTarget(c *C) { var ( - lk = NewLockKeeper() + lk = NewLockKeeper(getDownstreamMeta) task = "test-lock-keeper-multiple-target" source = "mysql-replica-1" upSchema = "foo" @@ -277,6 +292,11 @@ func (t *testKeeper) TestTableKeeper(c *C) { // no tables exist before Init/Update. c.Assert(tk.FindTables(task1, downSchema, downTable), IsNil) + for schema, tables := range tt11.UpTables { + for table := range tables { + c.Assert(tk.SourceTableExist(tt11.Task, tt11.Source, schema, table, downSchema, downTable), IsFalse) + } + } // Init with `nil` is fine. tk.Init(nil) @@ -288,6 +308,11 @@ func (t *testKeeper) TestTableKeeper(c *C) { c.Assert(tts, HasLen, 2) c.Assert(tts[0], DeepEquals, tt11) c.Assert(tts[1], DeepEquals, tt12) + for schema, tables := range tt11.UpTables { + for table := range tables { + c.Assert(tk.SourceTableExist(tt11.Task, tt11.Source, schema, table, downSchema, downTable), IsTrue) + } + } // adds new tables. c.Assert(tk.Update(st21), IsTrue) @@ -300,11 +325,21 @@ func (t *testKeeper) TestTableKeeper(c *C) { tts = tk.FindTables(task2, downSchema, downTable) c.Assert(tts, HasLen, 1) c.Assert(tts[0], DeepEquals, tt22) + for schema, tables := range tt22.UpTables { + for table := range tables { + c.Assert(tk.SourceTableExist(tt22.Task, tt22.Source, schema, table, downSchema, downTable), IsTrue) + } + } // deletes tables. st22.IsDeleted = true c.Assert(tk.Update(st22), IsTrue) c.Assert(tk.FindTables(task2, downSchema, downTable), IsNil) + for schema, tables := range tt22.UpTables { + for table := range tables { + c.Assert(tk.SourceTableExist(tt22.Task, tt22.Source, schema, table, downSchema, downTable), IsFalse) + } + } // try to delete, but not exist. c.Assert(tk.Update(st22), IsFalse) @@ -316,6 +351,11 @@ func (t *testKeeper) TestTableKeeper(c *C) { c.Assert(tts, HasLen, 2) c.Assert(tts[0], DeepEquals, tt11) c.Assert(tts[1], DeepEquals, tt12) + for schema, tables := range tt11.UpTables { + for table := range tables { + c.Assert(tk.SourceTableExist(tt11.Task, tt11.Source, schema, table, downSchema, downTable), IsTrue) + } + } // add a table for st11. c.Assert(tk.AddTable(task1, st11.Source, "db-2", "tbl-3", downSchema, downTable), IsTrue) @@ -346,6 +386,19 @@ func (t *testKeeper) TestTableKeeper(c *C) { c.Assert(tk.RemoveTable(task1, "not-exit", "db", "tbl-1", downSchema, downTable), IsFalse) tts = tk.FindTables(task1, downSchema, downTable) c.Assert(tts[1], DeepEquals, tt12) + + c.Assert(tk.RemoveTableByTask("hahaha"), IsFalse) + tk.RemoveTableByTaskAndSources("hahaha", nil) + tts = tk.FindTables(task1, downSchema, downTable) + c.Assert(tts, HasLen, 3) + tk.RemoveTableByTaskAndSources(task1, []string{"hahaha"}) + tts = tk.FindTables(task1, downSchema, downTable) + c.Assert(tts, HasLen, 3) + tk.RemoveTableByTaskAndSources(task1, []string{source1, source2}) + tts = tk.FindTables(task1, downSchema, downTable) + c.Assert(tts, HasLen, 1) + c.Assert(tts[0].Source, Equals, "new-source") + c.Assert(tts[0].UpTables["db-2"], HasKey, "tbl-3") } func (t *testKeeper) TestTargetTablesForTask(c *C) { @@ -414,72 +467,65 @@ func (t *testKeeper) TestTargetTablesForTask(c *C) { }) } -func (t *testKeeper) TestRebuildLocksAndTables(c *C) { - defer clearTestInfoOperation(c) +func getDownstreamMeta(string) (*config.DBConfig, string) { + return nil, "" +} + +func (t *testKeeper) TestGetDownstreamMeta(c *C) { var ( - lk = NewLockKeeper() - task = "task" - source1 = "mysql-replica-1" - source2 = "mysql-replica-2" - upSchema = "foo" - upTable = "bar" - downSchema = "db" - downTable = "tbl" - DDLs1 = []string{"ALTER TABLE bar ADD COLUMN c1 INT"} - DDLs2 = []string{"ALTER TABLE bar DROP COLUMN c1"} - p = parser.New() - se = mock.NewContext() - tblID int64 = 111 - ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY)`) - ti1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 INT)`) - ti2 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 INT, c2 INT)`) - ti3 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c2 INT)`) - - i11 = NewInfo(task, source1, upSchema, upTable, downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}) - i21 = NewInfo(task, source2, upSchema, upTable, downSchema, downTable, DDLs2, ti2, []*model.TableInfo{ti3}) - - tts = []TargetTable{ - newTargetTable(task, source1, downSchema, downTable, map[string]map[string]struct{}{upSchema: {upTable: struct{}{}}}), - newTargetTable(task, source2, downSchema, downTable, map[string]map[string]struct{}{upSchema: {upTable: struct{}{}}}), + task1 = "hahaha" + task2 = "hihihi" + task3 = "hehehe" + ) + getDownstreamMetaFunc := func(task string) (*config.DBConfig, string) { + switch task { + case task1, task2: + return &config.DBConfig{}, "meta" + default: + return nil, "" } + } - lockID = utils.GenDDLLockID(task, downSchema, downTable) + conn.InitMockDB(c) + lk := NewLockKeeper(getDownstreamMetaFunc) + c.Assert(lk.downstreamMetaMap, HasLen, 0) - ifm = map[string]map[string]map[string]map[string]Info{ - task: { - source1: {upSchema: {upTable: i11}}, - source2: {upSchema: {upTable: i21}}, - }, - } - colm = map[string]map[string]map[string]map[string]map[string]DropColumnStage{ - lockID: { - "c3": { - source1: {upSchema: {upTable: DropNotDone}}, - source2: {upSchema: {upTable: DropNotDone}}, - }, - }, - } - lockJoined = map[string]schemacmp.Table{ - lockID: schemacmp.Encode(ti2), - } - lockTTS = map[string][]TargetTable{ - lockID: tts, - } - ) + downstreamMeta, err := lk.getDownstreamMeta(task3) + c.Assert(downstreamMeta, IsNil) + c.Assert(terror.ErrMasterOptimisticDownstreamMetaNotFound.Equal(err), IsTrue) - lk.RebuildLocksAndTables(etcdTestCli, ifm, colm, lockJoined, lockTTS, nil) - locks := lk.Locks() - c.Assert(len(locks), Equals, 1) - lock, ok := locks[lockID] - c.Assert(ok, IsTrue) - cmp, err := lock.Joined().Compare(schemacmp.Encode(ti2)) + downstreamMeta, err = lk.getDownstreamMeta(task1) + c.Assert(err, IsNil) + c.Assert(lk.downstreamMetaMap, HasLen, 1) + c.Assert(downstreamMeta, Equals, lk.downstreamMetaMap[task1]) + downstreamMeta2, err := lk.getDownstreamMeta(task1) c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) - cmp, err = lock.tables[source1][upSchema][upTable].Compare(schemacmp.Encode(ti0)) + c.Assert(lk.downstreamMetaMap, HasLen, 1) + c.Assert(downstreamMeta, Equals, downstreamMeta2) + + downstreamMeta3, err := lk.getDownstreamMeta(task2) c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) - cmp, err = lock.tables[source2][upSchema][upTable].Compare(schemacmp.Encode(ti2)) + c.Assert(lk.downstreamMetaMap, HasLen, 2) + c.Assert(lk.downstreamMetaMap, HasKey, task1) + c.Assert(lk.downstreamMetaMap, HasKey, task2) + c.Assert(downstreamMeta3, Equals, lk.downstreamMetaMap[task2]) + + lk.RemoveDownstreamMeta(task3) + c.Assert(lk.downstreamMetaMap, HasLen, 2) + c.Assert(lk.downstreamMetaMap, HasKey, task1) + c.Assert(lk.downstreamMetaMap, HasKey, task2) + + lk.RemoveDownstreamMeta(task1) + c.Assert(lk.downstreamMetaMap, HasLen, 1) + c.Assert(lk.downstreamMetaMap, HasKey, task2) + c.Assert(downstreamMeta3, Equals, lk.downstreamMetaMap[task2]) + + downstreamMeta, err = lk.getDownstreamMeta(task1) c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) - c.Assert(lock.columns, DeepEquals, colm[lockID]) + c.Assert(lk.downstreamMetaMap, HasLen, 2) + c.Assert(downstreamMeta, Equals, lk.downstreamMetaMap[task1]) + c.Assert(downstreamMeta3, Equals, lk.downstreamMetaMap[task2]) + + lk.Clear() + c.Assert(lk.downstreamMetaMap, HasLen, 0) } diff --git a/dm/pkg/shardddl/optimism/lock.go b/dm/pkg/shardddl/optimism/lock.go index 7b9740ce126..d1b207a9f48 100644 --- a/dm/pkg/shardddl/optimism/lock.go +++ b/dm/pkg/shardddl/optimism/lock.go @@ -14,16 +14,23 @@ package optimism import ( + "bytes" + "encoding/json" "fmt" "sync" + "github.com/pingcap/tidb-tools/pkg/dbutil" "github.com/pingcap/tidb-tools/pkg/schemacmp" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/parser/model" "go.etcd.io/etcd/clientv3" "go.uber.org/zap" + "golang.org/x/net/context" "github.com/pingcap/tiflow/dm/dm/master/metrics" + "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/pkg/cputil" "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/terror" ) @@ -75,30 +82,94 @@ type Lock struct { // record the partially dropped columns // column name -> source -> upSchema -> upTable -> int columns map[string]map[string]map[string]map[string]DropColumnStage + + downstreamMeta *DownstreamMeta } // NewLock creates a new Lock instance. -// NOTE: we MUST give the initial table info when creating the lock now. -func NewLock(cli *clientv3.Client, id, task, downSchema, downTable string, joined schemacmp.Table, tts []TargetTable) *Lock { +func NewLock(cli *clientv3.Client, id, task, downSchema, downTable string, joined schemacmp.Table, tts []TargetTable, downstreamMeta *DownstreamMeta) *Lock { l := &Lock{ - cli: cli, - ID: id, - Task: task, - DownSchema: downSchema, - DownTable: downTable, - joined: joined, - tables: make(map[string]map[string]map[string]schemacmp.Table), - done: make(map[string]map[string]map[string]bool), - synced: true, - versions: make(map[string]map[string]map[string]int64), - columns: make(map[string]map[string]map[string]map[string]DropColumnStage), + cli: cli, + ID: id, + Task: task, + DownSchema: downSchema, + DownTable: downTable, + joined: joined, + tables: make(map[string]map[string]map[string]schemacmp.Table), + done: make(map[string]map[string]map[string]bool), + synced: true, + versions: make(map[string]map[string]map[string]int64), + columns: make(map[string]map[string]map[string]map[string]DropColumnStage), + downstreamMeta: downstreamMeta, } l.addTables(tts) metrics.ReportDDLPending(task, metrics.DDLPendingNone, metrics.DDLPendingSynced) - + // pre join because tables may have different schema at the beginning + l.joinTable() return l } +// FetchTableInfos fetch all table infos for a lock. +func (l *Lock) FetchTableInfos(task, source, schema, table string) (*model.TableInfo, error) { + if l.downstreamMeta == nil { + return nil, terror.ErrMasterOptimisticDownstreamMetaNotFound.Generate(task) + } + + db, err := conn.DefaultDBProvider.Apply(l.downstreamMeta.dbConfig) + if err != nil { + return nil, err + } + defer db.Close() + + ctx, cancel := context.WithTimeout(context.Background(), dbutil.DefaultTimeout) + defer cancel() + + query := `SELECT table_info FROM ` + dbutil.TableName(l.downstreamMeta.meta, cputil.SyncerCheckpoint(task)) + ` WHERE id = ? AND cp_schema = ? AND cp_table = ?` + row := db.DB.QueryRowContext(ctx, query, source, schema, table) + if row.Err() != nil { + return nil, terror.ErrDBExecuteFailed.Delegate(row.Err(), query) + } + var tiBytes []byte + if err := row.Scan(&tiBytes); err != nil { + return nil, terror.ErrDBExecuteFailed.Delegate(err, query) + } + var ti *model.TableInfo + if bytes.Equal(tiBytes, []byte("null")) { + log.L().Warn("null table info", zap.String("query", query), zap.String("source", source), zap.String("schema", schema), zap.String("table", table)) + return nil, terror.ErrMasterOptimisticDownstreamMetaNotFound.Generate(task) + } + if err := json.Unmarshal(tiBytes, &ti); err != nil { + return nil, err + } + return ti, nil +} + +// joinTable join tables for a lock and update l.joined. +func (l *Lock) joinTable() { + var ( + joined = l.joined + firstTable = true + ) + for _, schemaTables := range l.tables { + for _, tables := range schemaTables { + for _, ti := range tables { + if firstTable { + joined = ti + firstTable = false + } else { + newJoined, err := joined.Join(ti) + if err != nil { + log.L().Error(fmt.Sprintf("fail to join table info %s with %s", joined, ti), zap.String("lockID", l.ID), log.ShortError(err)) + return + } + joined = newJoined + } + } + } + } + l.joined = joined +} + // TrySync tries to sync the lock, re-entrant. // new upstream sources may join when the DDL lock is in syncing, // so we need to merge these new sources. @@ -398,6 +469,56 @@ func (l *Lock) TryRemoveTable(source, schema, table string) bool { return true } +// TryRemoveTable tries to remove tables in the lock by sources. +// return drop columns for later use. +func (l *Lock) TryRemoveTableBySources(sources []string) []string { + l.mu.Lock() + defer l.mu.Unlock() + + // record drop columns for sources + dropColumns := make([]string, 0) + for col, sourceColumns := range l.columns { + for _, source := range sources { + if _, ok := sourceColumns[source]; ok { + dropColumns = append(dropColumns, col) + break + } + } + } + + for _, source := range sources { + if _, ok := l.tables[source]; !ok { + continue + } + + delete(l.tables, source) + _, remain := l.syncStatus() + l.synced = remain == 0 + delete(l.done, source) + delete(l.versions, source) + for _, sourceColumns := range l.columns { + delete(sourceColumns, source) + } + log.L().Info("tables removed from the lock", zap.String("lock", l.ID), zap.String("source", source)) + } + return dropColumns +} + +// HasTables check whether a lock has tables. +func (l *Lock) HasTables() bool { + l.mu.Lock() + defer l.mu.Unlock() + + for _, schemas := range l.tables { + for _, tables := range schemas { + for range tables { + return true + } + } + } + return false +} + // IsSynced returns whether the lock has synced. // In the optimistic mode, we call it `synced` if table info of all tables are the same, // and we define `remain` as the table count which have different table info with the joined one, @@ -533,6 +654,7 @@ func (l *Lock) tryRevertDone(source, schema, table string) { } // addTables adds any not-existing tables into the lock. +// For a new table, try to fetch table info from downstream. func (l *Lock) addTables(tts []TargetTable) { for _, tt := range tts { if _, ok := l.tables[tt.Source]; !ok { @@ -548,7 +670,15 @@ func (l *Lock) addTables(tts []TargetTable) { } for table := range tables { if _, ok := l.tables[tt.Source][schema][table]; !ok { - l.tables[tt.Source][schema][table] = l.joined + ti, err := l.FetchTableInfos(tt.Task, tt.Source, schema, table) + if err != nil { + log.L().Error("source table info not found, use joined table info instead", zap.String("task", tt.Task), zap.String("source", tt.Source), zap.String("schema", schema), zap.String("table", table), log.ShortError(err)) + l.tables[tt.Source][schema][table] = l.joined + } else { + t := schemacmp.Encode(ti) + log.L().Debug("get source table info", zap.String("task", tt.Task), zap.String("source", tt.Source), zap.String("schema", schema), zap.String("table", table), zap.Stringer("info", t)) + l.tables[tt.Source][schema][table] = t + } l.done[tt.Source][schema][table] = false l.versions[tt.Source][schema][table] = 0 log.L().Info("table added to the lock", zap.String("lock", l.ID), diff --git a/dm/pkg/shardddl/optimism/lock_test.go b/dm/pkg/shardddl/optimism/lock_test.go index 1ed4ebc64a0..5960b139491 100644 --- a/dm/pkg/shardddl/optimism/lock_test.go +++ b/dm/pkg/shardddl/optimism/lock_test.go @@ -14,8 +14,11 @@ package optimism import ( + "encoding/json" + "fmt" "testing" + "github.com/DATA-DOG/go-sqlmock" . "github.com/pingcap/check" "github.com/pingcap/tidb-tools/pkg/schemacmp" "github.com/pingcap/tidb/parser" @@ -24,6 +27,9 @@ import ( "github.com/pingcap/tidb/util/mock" "go.etcd.io/etcd/integration" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/pkg/cputil" "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/terror" "github.com/pingcap/tiflow/dm/pkg/utils" @@ -83,7 +89,7 @@ func (t *testLock) TestLockTrySyncNormal(c *C) { newTargetTable(task, sources[1], downSchema, downTable, tables), } - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ sources[0]: { @@ -385,7 +391,7 @@ func (t *testLock) TestLockTrySyncIndex(c *C) { newTargetTable(task, source, downSchema, downTable, tables), } - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -469,7 +475,7 @@ func (t *testLock) TestLockTrySyncNullNotNull(c *C) { newTargetTable(task, source, downSchema, downTable, tables), } - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -540,7 +546,7 @@ func (t *testLock) TestLockTrySyncIntBigint(c *C) { newTargetTable(task, source, downSchema, downTable, tables), } - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -593,7 +599,7 @@ func (t *testLock) TestLockTrySyncNoDiff(c *C) { newTargetTable(task, source, downSchema, downTable, tables), } - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -636,7 +642,7 @@ func (t *testLock) TestLockTrySyncNewTable(c *C) { tables = map[string]map[string]struct{}{db1: {tbl1: struct{}{}}} tts = []TargetTable{newTargetTable(task, source1, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source1: { db1: {tbl1: 0}, @@ -749,7 +755,7 @@ func (t *testLock) TestLockTrySyncRevert(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -914,7 +920,7 @@ func (t *testLock) TestLockTrySyncConflictNonIntrusive(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -1036,7 +1042,7 @@ func (t *testLock) TestLockTrySyncConflictIntrusive(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -1224,7 +1230,7 @@ func (t *testLock) TestLockTrySyncMultipleChangeDDL(c *C) { newTargetTable(task, sources[1], downSchema, downTable, tables), } - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ sources[0]: { @@ -1367,7 +1373,7 @@ func (t *testLock) TestTryRemoveTable(c *C) { tables = map[string]map[string]struct{}{db: {tbl1: struct{}{}, tbl2: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -1437,6 +1443,103 @@ func (t *testLock) TestTryRemoveTable(c *C) { c.Assert(l.TryRemoveTable("not-exist", db, tbl1), IsFalse) } +func (t *testLock) TestTryRemoveTableWithSources(c *C) { + var ( + ID = "test_lock_try_remove_table-`foo`.`bar`" + task = "test_lock_try_remove_table" + source1 = "mysql-replica-1" + source2 = "mysql-replica-2" + downSchema = "foo" + downTable = "bar" + db = "foo" + tbl1 = "bar1" + tbl2 = "bar2" + p = parser.New() + se = mock.NewContext() + tblID int64 = 111 + DDLs1 = []string{"ALTER TABLE bar DROP COLUMN c1"} + ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 INT)`) + ti1 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY)`) + + tables = map[string]map[string]struct{}{db: {tbl1: struct{}{}, tbl2: struct{}{}}} + tts = []TargetTable{newTargetTable(task, source1, downSchema, downTable, tables), newTargetTable(task, source2, downSchema, downTable, tables)} + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) + + vers = map[string]map[string]map[string]int64{ + source1: { + db: {tbl1: 0, tbl2: 0}, + }, + source2: { + db: {tbl1: 0, tbl2: 0}, + }, + } + ) + + // only one table exists before TrySync. + t.checkLockSynced(c, l) + t.checkLockNoDone(c, l) + + // TrySync for the first table. + info := newInfoWithVersion(task, source1, db, tbl1, downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) + DDLs, cols, err := l.TrySync(info, tts) + c.Assert(err, IsNil) + c.Assert(DDLs, DeepEquals, []string{}) + c.Assert(cols, DeepEquals, []string{"c1"}) + c.Assert(l.versions, DeepEquals, vers) + ready := l.Ready() + c.Assert(ready, HasLen, 2) + c.Assert(ready[source1], HasLen, 1) + c.Assert(ready[source1][db], HasLen, 2) + c.Assert(ready[source1][db][tbl1], IsFalse) + c.Assert(ready[source1][db][tbl2], IsTrue) + c.Assert(ready[source2], HasLen, 1) + c.Assert(ready[source2][db], HasLen, 2) + c.Assert(ready[source2][db][tbl1], IsTrue) + c.Assert(ready[source2][db][tbl2], IsTrue) + + // TryRemoveTableBySources with nil + c.Assert(len(l.TryRemoveTableBySources(nil)), Equals, 0) + ready = l.Ready() + c.Assert(ready, HasLen, 2) + + // TryRemoveTableBySources with wrong source + tts = tts[:1] + c.Assert(len(l.TryRemoveTableBySources([]string{"hahaha"})), Equals, 0) + ready = l.Ready() + c.Assert(ready, HasLen, 2) + + // TryRemoveTableBySources with source2 + c.Assert(len(l.TryRemoveTableBySources([]string{source2})), Equals, 0) + ready = l.Ready() + c.Assert(ready, HasLen, 1) + c.Assert(ready[source1], HasLen, 1) + c.Assert(ready[source1][db], HasLen, 2) + c.Assert(ready[source1][db][tbl1], IsFalse) + c.Assert(ready[source1][db][tbl2], IsTrue) + delete(vers, source2) + c.Assert(l.versions, DeepEquals, vers) + c.Assert(l.HasTables(), IsTrue) + + // TrySync with second table + info = newInfoWithVersion(task, source1, db, tbl2, downSchema, downTable, DDLs1, ti0, []*model.TableInfo{ti1}, vers) + DDLs, cols, err = l.TrySync(info, tts) + c.Assert(err, IsNil) + c.Assert(DDLs, DeepEquals, DDLs1) + c.Assert(cols, DeepEquals, []string{"c1"}) + c.Assert(l.versions, DeepEquals, vers) + ready = l.Ready() + c.Assert(ready, HasLen, 1) + c.Assert(ready[source1], HasLen, 1) + c.Assert(ready[source1][db], HasLen, 2) + c.Assert(ready[source1][db][tbl1], IsTrue) + c.Assert(ready[source1][db][tbl2], IsTrue) + + // TryRemoveTableBySources with source1,source2 + cols = l.TryRemoveTableBySources([]string{source1}) + c.Assert(cols, DeepEquals, []string{"c1"}) + c.Assert(l.HasTables(), IsFalse) +} + func (t *testLock) TestLockTryMarkDone(c *C) { var ( ID = "test_lock_try_mark_done-`foo`.`bar`" @@ -1459,7 +1562,7 @@ func (t *testLock) TestLockTryMarkDone(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -1567,7 +1670,7 @@ func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -1609,7 +1712,7 @@ func (t *testLock) TestAddDifferentFieldLenColumns(c *C) { c.Assert(l.versions, DeepEquals, vers) // case 2: add a column with a smaller field length - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) // TrySync for the first table, no table has done the DDLs operation. vers[source][db][tbls[0]]-- @@ -1657,7 +1760,7 @@ func (t *testLock) TestAddNotFullyDroppedColumns(c *C) { tables = map[string]map[string]struct{}{db: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}} tts = []TargetTable{newTargetTable(task, source, downSchema, downTable, tables)} - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -1853,7 +1956,7 @@ func (t *testLock) TestLockTrySyncDifferentIndex(c *C) { newTargetTable(task, source, downSchema, downTable, tables), } - l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts) + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) vers = map[string]map[string]map[string]int64{ source: { @@ -1909,3 +2012,62 @@ func (t *testLock) TestLockTrySyncDifferentIndex(c *C) { c.Assert(l.versions, DeepEquals, vers) t.checkLockSynced(c, l) } + +func (t *testLock) TestFetchTableInfo(c *C) { + var ( + meta = "meta" + ID = "test_lock_try_sync_index-`foo`.`bar`" + task = "test_lock_try_sync_index" + source = "mysql-replica-1" + downSchema = "db" + downTable = "bar" + schema = "db" + tbls = []string{"bar1", "bar2"} + p = parser.New() + se = mock.NewContext() + tblID int64 = 111 + ti0 = createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY, c1 INT, UNIQUE INDEX idx_c1(c1))`) + tables = map[string]map[string]struct{}{ + schema: {tbls[0]: struct{}{}, tbls[1]: struct{}{}}, + } + tts = []TargetTable{ + newTargetTable(task, source, downSchema, downTable, tables), + } + query = fmt.Sprintf("SELECT table_info FROM `%s`.`%s` WHERE id = \\? AND cp_schema = \\? AND cp_table = \\?", meta, cputil.SyncerCheckpoint(task)) + ) + + // nil downstream meta + l := NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, nil) + ti, err := l.FetchTableInfos(task, source, schema, tbls[0]) + c.Assert(terror.ErrMasterOptimisticDownstreamMetaNotFound.Equal(err), IsTrue) + c.Assert(ti, IsNil) + + // table info not exist + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, &DownstreamMeta{dbConfig: &config.DBConfig{}, meta: meta}) + conn.DefaultDBProvider = &conn.DefaultDBProviderImpl{} + mock := conn.InitMockDB(c) + mock.ExpectQuery(query).WithArgs(source, schema, tbls[0]).WillReturnRows(sqlmock.NewRows([]string{"table_info"})) + ti, err = l.FetchTableInfos(task, source, schema, tbls[0]) + c.Assert(terror.ErrDBExecuteFailed.Equal(err), IsTrue) + c.Assert(ti, IsNil) + + // null table info + l = NewLock(etcdTestCli, ID, task, downSchema, downTable, schemacmp.Encode(ti0), tts, &DownstreamMeta{dbConfig: &config.DBConfig{}, meta: meta}) + conn.DefaultDBProvider = &conn.DefaultDBProviderImpl{} + mock = conn.InitMockDB(c) + mock.ExpectQuery(query).WithArgs(source, schema, tbls[0]).WillReturnRows(sqlmock.NewRows([]string{"table_info"}).AddRow("null")) + ti, err = l.FetchTableInfos(task, source, schema, tbls[0]) + c.Assert(terror.ErrMasterOptimisticDownstreamMetaNotFound.Equal(err), IsTrue) + c.Assert(ti, IsNil) + + // succeed + tiBytes, err := json.Marshal(ti0) + c.Assert(err, IsNil) + conn.DefaultDBProvider = &conn.DefaultDBProviderImpl{} + mock = conn.InitMockDB(c) + mock.ExpectQuery(query).WithArgs(source, schema, tbls[0]).WillReturnRows(sqlmock.NewRows([]string{"table_info"}).AddRow(tiBytes)) + ti, err = l.FetchTableInfos(task, source, schema, tbls[0]) + c.Assert(err, IsNil) + c.Assert(mock.ExpectationsWereMet(), IsNil) + c.Assert(ti, DeepEquals, ti0) +} diff --git a/dm/pkg/shardddl/optimism/ops.go b/dm/pkg/shardddl/optimism/ops.go index 7cd13bace1d..28c92f3e495 100644 --- a/dm/pkg/shardddl/optimism/ops.go +++ b/dm/pkg/shardddl/optimism/ops.go @@ -18,7 +18,6 @@ import ( "github.com/pingcap/tiflow/dm/dm/common" "github.com/pingcap/tiflow/dm/pkg/etcdutil" - "github.com/pingcap/tiflow/dm/pkg/utils" ) // PutSourceTablesInfo puts source tables and a shard DDL info. @@ -48,10 +47,10 @@ func PutSourceTablesDeleteInfo(cli *clientv3.Client, st SourceTables, info Info) return rev, err } -// DeleteInfosOperationsSchemaColumn deletes the shard DDL infos, operations, init schemas and dropped columns in etcd. +// DeleteInfosOperationsColumns deletes the shard DDL infos, operations, and dropped columns in etcd. // This function should often be called by DM-master when removing the lock. // Only delete when all info's version are greater or equal to etcd's version, otherwise it means new info was putted into etcd before. -func DeleteInfosOperationsSchemaColumn(cli *clientv3.Client, infos []Info, ops []Operation, schema InitSchema) (int64, bool, error) { +func DeleteInfosOperationsColumns(cli *clientv3.Client, infos []Info, ops []Operation, lockID string) (int64, bool, error) { opsDel := make([]clientv3.Op, 0, len(infos)+len(ops)) cmps := make([]clientv3.Cmp, 0, len(infos)) for _, info := range infos { @@ -62,8 +61,7 @@ func DeleteInfosOperationsSchemaColumn(cli *clientv3.Client, infos []Info, ops [ for _, op := range ops { opsDel = append(opsDel, deleteOperationOp(op)) } - opsDel = append(opsDel, deleteInitSchemaOp(schema.Task, schema.DownSchema, schema.DownTable)) - opsDel = append(opsDel, deleteDroppedColumnsByLockOp(utils.GenDDLLockID(schema.Task, schema.DownSchema, schema.DownTable))) + opsDel = append(opsDel, deleteDroppedColumnsByLockOp(lockID)) resp, rev, err := etcdutil.DoOpsInOneCmpsTxnWithRetry(cli, cmps, opsDel, []clientv3.Op{}) if err != nil { return 0, false, err @@ -71,16 +69,36 @@ func DeleteInfosOperationsSchemaColumn(cli *clientv3.Client, infos []Info, ops [ return rev, resp.Succeeded, nil } -// DeleteInfosOperationsTablesSchemasByTask deletes the shard DDL infos and operations in etcd. -func DeleteInfosOperationsTablesSchemasByTask(cli *clientv3.Client, task string, lockIDSet map[string]struct{}) (int64, error) { +// DeleteInfosOperationsTablesByTask deletes the shard DDL infos and operations in etcd. +// This function should often be called by DM-master when stop a task for all sources. +func DeleteInfosOperationsTablesByTask(cli *clientv3.Client, task string, lockIDSet map[string]struct{}) (int64, error) { opsDel := make([]clientv3.Op, 0, 5) opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismInfoKeyAdapter.Encode(task), clientv3.WithPrefix())) opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismOperationKeyAdapter.Encode(task), clientv3.WithPrefix())) opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismSourceTablesKeyAdapter.Encode(task), clientv3.WithPrefix())) - opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismInitSchemaKeyAdapter.Encode(task), clientv3.WithPrefix())) for lockID := range lockIDSet { opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(lockID), clientv3.WithPrefix())) } _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, opsDel...) return rev, err } + +// DeleteInfosOperationsTablesByTaskAndSource deletes the shard DDL infos and operations in etcd by task and source. +// This function should often be called by DM-master when stop a task for sources. +func DeleteInfosOperationsTablesByTaskAndSource(cli *clientv3.Client, task string, sources []string, dropColumns map[string][]string) (int64, error) { + opsDel := make([]clientv3.Op, 0, 5) + for _, source := range sources { + opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismInfoKeyAdapter.Encode(task, source), clientv3.WithPrefix())) + opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismOperationKeyAdapter.Encode(task, source), clientv3.WithPrefix())) + opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismSourceTablesKeyAdapter.Encode(task, source), clientv3.WithPrefix())) + for lockID, cols := range dropColumns { + for _, col := range cols { + for _, source := range sources { + opsDel = append(opsDel, clientv3.OpDelete(common.ShardDDLOptimismDroppedColumnsKeyAdapter.Encode(lockID, col, source), clientv3.WithPrefix())) + } + } + } + } + _, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, opsDel...) + return rev, err +} diff --git a/dm/pkg/shardddl/optimism/ops_test.go b/dm/pkg/shardddl/optimism/ops_test.go index 55270f0d5e8..da140184268 100644 --- a/dm/pkg/shardddl/optimism/ops_test.go +++ b/dm/pkg/shardddl/optimism/ops_test.go @@ -30,7 +30,6 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { DDLs = []string{"ALTER TABLE bar ADD COLUMN c1 INT"} info = NewInfo(task, source, upSchema, upTable, downSchema, downTable, DDLs, nil, nil) op = NewOperation("test-ID", task, source, upSchema, upTable, DDLs, ConflictResolved, "", false, []string{}) - is = NewInitSchema(task, downSchema, downTable, nil) ) // put info. @@ -52,15 +51,8 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { c.Assert(opm, HasLen, 1) c.Assert(opm[task][source][upSchema][upTable], DeepEquals, op) - // put init schema. - _, _, err = PutInitSchemaIfNotExist(etcdTestCli, is) - c.Assert(err, IsNil) - isc, _, err := GetInitSchema(etcdTestCli, is.Task, is.DownSchema, is.DownTable) - c.Assert(err, IsNil) - c.Assert(isc, DeepEquals, is) - // DELETE info and operation with version 0 - _, deleted, err := DeleteInfosOperationsSchemaColumn(etcdTestCli, []Info{info}, []Operation{op}, is) + _, deleted, err := DeleteInfosOperationsColumns(etcdTestCli, []Info{info}, []Operation{op}, genDDLLockID(info)) c.Assert(err, IsNil) c.Assert(deleted, IsFalse) @@ -71,12 +63,9 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { opm, _, err = GetAllOperations(etcdTestCli) c.Assert(err, IsNil) c.Assert(opm, HasLen, 1) - isc, _, err = GetInitSchema(etcdTestCli, is.Task, is.DownSchema, is.DownTable) - c.Assert(err, IsNil) - c.Assert(isc.IsEmpty(), IsFalse) // DELETE info and operation with version 1 - _, deleted, err = DeleteInfosOperationsSchemaColumn(etcdTestCli, []Info{infoWithVer}, []Operation{op}, is) + _, deleted, err = DeleteInfosOperationsColumns(etcdTestCli, []Info{infoWithVer}, []Operation{op}, genDDLLockID(infoWithVer)) c.Assert(err, IsNil) c.Assert(deleted, IsTrue) @@ -87,9 +76,6 @@ func (t *testForEtcd) TestDeleteInfosOperationsSchema(c *C) { opm, _, err = GetAllOperations(etcdTestCli) c.Assert(err, IsNil) c.Assert(opm, HasLen, 0) - isc, _, err = GetInitSchema(etcdTestCli, is.Task, is.DownSchema, is.DownTable) - c.Assert(err, IsNil) - c.Assert(isc.IsEmpty(), IsTrue) } func (t *testForEtcd) TestSourceTablesInfo(c *C) { diff --git a/dm/pkg/shardddl/optimism/schema.go b/dm/pkg/shardddl/optimism/schema.go deleted file mode 100644 index c4f4167e4df..00000000000 --- a/dm/pkg/shardddl/optimism/schema.go +++ /dev/null @@ -1,151 +0,0 @@ -// Copyright 2020 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 optimism - -import ( - "encoding/json" - - "github.com/pingcap/tidb/parser/model" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/clientv3util" - - "github.com/pingcap/tiflow/dm/dm/common" - "github.com/pingcap/tiflow/dm/pkg/etcdutil" -) - -// InitSchema represents the initial schema (schema before the lock constructed) of a merged table. -// NOTE: `Task`, `DownSchema` and `DownTable` are redundant in the etcd key path for convenient. -type InitSchema struct { - Task string `json:"task"` // data migration task name - DownSchema string `json:"down-schema"` // downstream/target schema name - DownTable string `json:"down-table"` // downstream/target table name - TableInfo *model.TableInfo `json:"table-info"` // the initial table info (schema) -} - -// NewInitSchema creates a new InitSchema instance. -func NewInitSchema(task, downSchema, downTable string, tableInfo *model.TableInfo) InitSchema { - return InitSchema{ - Task: task, - DownSchema: downSchema, - DownTable: downTable, - TableInfo: tableInfo, - } -} - -// String implements Stringer interface. -func (is InitSchema) String() string { - s, _ := is.toJSON() - return s -} - -// toJSON returns the string of JSON represent. -func (is InitSchema) toJSON() (string, error) { - data, err := json.Marshal(is) - if err != nil { - return "", err - } - return string(data), nil -} - -// IsEmpty returns true when this InitSchema has no value. -func (is InitSchema) IsEmpty() bool { - var emptyIS InitSchema - return is == emptyIS -} - -// infoFromJSON constructs InitSchema from its JSON represent. -func initSchemaFromJSON(s string) (is InitSchema, err error) { - err = json.Unmarshal([]byte(s), &is) - return -} - -// GetInitSchema gets the InitSchema for the specified downstream table. -func GetInitSchema(cli *clientv3.Client, task, downSchema, downTable string) (InitSchema, int64, error) { - var is InitSchema - op := clientv3.OpGet(common.ShardDDLOptimismInitSchemaKeyAdapter.Encode(task, downSchema, downTable)) - respTxn, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, op) - if err != nil { - return is, 0, err - } - resp := respTxn.Responses[0].GetResponseRange() - - if resp.Count > 0 { - is, err = initSchemaFromJSON(string(resp.Kvs[0].Value)) - if err != nil { - return is, 0, err - } - } - return is, rev, nil -} - -// GetAllInitSchemas gets all init schemas from etcd. -// This function should often be called by DM-master. -// k/k/k/v: task-name -> downstream-schema-name -> downstream-table-name -> InitSchema. -func GetAllInitSchemas(cli *clientv3.Client) (map[string]map[string]map[string]InitSchema, int64, error) { - initSchemas := make(map[string]map[string]map[string]InitSchema) - op := clientv3.OpGet(common.ShardDDLOptimismInitSchemaKeyAdapter.Path(), clientv3.WithPrefix()) - respTxn, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, op) - if err != nil { - return nil, 0, err - } - resp := respTxn.Responses[0].GetResponseRange() - - for _, kv := range resp.Kvs { - schema, err := initSchemaFromJSON(string(kv.Value)) - if err != nil { - return nil, 0, err - } - if _, ok := initSchemas[schema.Task]; !ok { - initSchemas[schema.Task] = make(map[string]map[string]InitSchema) - } - if _, ok := initSchemas[schema.Task][schema.DownSchema]; !ok { - initSchemas[schema.Task][schema.DownSchema] = make(map[string]InitSchema) - } - initSchemas[schema.Task][schema.DownSchema][schema.DownTable] = schema - } - return initSchemas, rev, nil -} - -// PutInitSchemaIfNotExist puts the InitSchema into ectd if no previous one exists. -func PutInitSchemaIfNotExist(cli *clientv3.Client, is InitSchema) (rev int64, putted bool, err error) { - value, err := is.toJSON() - if err != nil { - return 0, false, err - } - key := common.ShardDDLOptimismInitSchemaKeyAdapter.Encode(is.Task, is.DownSchema, is.DownTable) - - cmp := clientv3util.KeyMissing(key) - op := clientv3.OpPut(key, value) - - resp, rev, err := etcdutil.DoOpsInOneCmpsTxnWithRetry(cli, []clientv3.Cmp{cmp}, []clientv3.Op{op}, []clientv3.Op{}) - if err != nil { - return 0, false, err - } - return rev, resp.Succeeded, nil -} - -// DeleteInitSchema tries to delete the InitSchema for the specified downstream table. -func DeleteInitSchema(cli *clientv3.Client, task, downSchema, downTable string) (rev int64, deleted bool, err error) { - op := deleteInitSchemaOp(task, downSchema, downTable) - resp, rev, err := etcdutil.DoOpsInOneTxnWithRetry(cli, op) - if err != nil { - return 0, false, err - } - return rev, resp.Succeeded, nil -} - -// deleteInitSchemaOp returns a DELETE etcd operation for init schema. -func deleteInitSchemaOp(task, downSchema, downTable string) clientv3.Op { - return clientv3.OpDelete(common.ShardDDLOptimismInitSchemaKeyAdapter.Encode(task, downSchema, downTable)) -} diff --git a/dm/pkg/shardddl/optimism/schema_test.go b/dm/pkg/shardddl/optimism/schema_test.go deleted file mode 100644 index f5f3ba582a4..00000000000 --- a/dm/pkg/shardddl/optimism/schema_test.go +++ /dev/null @@ -1,109 +0,0 @@ -// Copyright 2020 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 optimism - -import ( - . "github.com/pingcap/check" - "github.com/pingcap/tidb/parser" - "github.com/pingcap/tidb/util/mock" -) - -func (t *testForEtcd) TestInitSchemaJSON(c *C) { - is1 := NewInitSchema("test", "foo", "bar", nil) - j, err := is1.toJSON() - c.Assert(err, IsNil) - c.Assert(j, Equals, `{"task":"test","down-schema":"foo","down-table":"bar","table-info":null}`) - c.Assert(j, Equals, is1.String()) - - is2, err := initSchemaFromJSON(j) - c.Assert(err, IsNil) - c.Assert(is2, DeepEquals, is1) -} - -func (t *testForEtcd) TestInitSchemaEtcd(c *C) { - defer clearTestInfoOperation(c) - - var ( - task = "test-init-schema-etcd" - downSchema = "foo" - downTable = "bar" - downTable2 = "bar2" - p = parser.New() - se = mock.NewContext() - tblID int64 = 111 - tblI1 = createTableInfo(c, p, se, tblID, "CREATE TABLE bar (id INT PRIMARY KEY)") - tblI2 = createTableInfo(c, p, se, tblID, "CREATE TABLE bar (id INT PRIMARY KEY, c1 INT)") - tblI3 = createTableInfo(c, p, se, tblID, "CREATE TABLE bar2 (id INT PRIMARY KEY, c INT)") - is1 = NewInitSchema(task, downSchema, downTable, tblI1) - is2 = NewInitSchema(task, downSchema, downTable, tblI2) - is3 = NewInitSchema(task, downSchema, downTable2, tblI3) - ) - - // try to get, but no one exists. - isc, rev0, err := GetInitSchema(etcdTestCli, task, downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(rev0, Greater, int64(0)) - c.Assert(isc.IsEmpty(), IsTrue) - - // put the init schema. - rev1, putted, err := PutInitSchemaIfNotExist(etcdTestCli, is1) - c.Assert(err, IsNil) - c.Assert(rev1, Greater, rev0) - c.Assert(putted, IsTrue) - - // get it back. - isc, rev2, err := GetInitSchema(etcdTestCli, task, downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(rev2, Equals, rev1) - c.Assert(isc, DeepEquals, is1) - - // can't put again if a previous one exist. - rev3, putted, err := PutInitSchemaIfNotExist(etcdTestCli, is1) - c.Assert(err, IsNil) - c.Assert(rev3, Equals, rev1) - c.Assert(putted, IsFalse) - rev3, putted, err = PutInitSchemaIfNotExist(etcdTestCli, is2) - c.Assert(err, IsNil) - c.Assert(rev3, Equals, rev1) - c.Assert(putted, IsFalse) - - // put new init schema with different downstream info. - rev4, putted, err := PutInitSchemaIfNotExist(etcdTestCli, is3) - c.Assert(err, IsNil) - c.Assert(rev4, Greater, rev3) - c.Assert(putted, IsTrue) - - // get all init schemas. - initSchemas, rev5, err := GetAllInitSchemas(etcdTestCli) - c.Assert(err, IsNil) - c.Assert(rev5, Equals, rev4) - c.Assert(initSchemas[is1.Task][is1.DownSchema][is1.DownTable], DeepEquals, is1) - c.Assert(initSchemas[is3.Task][is3.DownSchema][is3.DownTable], DeepEquals, is3) - - // delete the schema. - rev6, deleted, err := DeleteInitSchema(etcdTestCli, is1.Task, is1.DownSchema, is1.DownTable) - c.Assert(err, IsNil) - c.Assert(rev6, Greater, rev5) - c.Assert(deleted, IsTrue) - rev7, deleted, err := DeleteInitSchema(etcdTestCli, is3.Task, is3.DownSchema, is3.DownTable) - c.Assert(err, IsNil) - c.Assert(rev7, Greater, rev6) - c.Assert(deleted, IsTrue) - - // not exist now. - initSchemas, rev8, err := GetAllInitSchemas(etcdTestCli) - c.Assert(err, IsNil) - c.Assert(rev8, Equals, rev7) - c.Assert(initSchemas, HasLen, 0) -} diff --git a/dm/pkg/terror/error_list.go b/dm/pkg/terror/error_list.go index 5d92eff424a..6c95627d941 100644 --- a/dm/pkg/terror/error_list.go +++ b/dm/pkg/terror/error_list.go @@ -247,6 +247,8 @@ const ( codeConfigOpenAPITaskConfigExist codeConfigOpenAPITaskConfigNotExist codeCollationCompatibleNotSupport + codeConfigInvalidLoadMode + codeConfigInvalidLoadDuplicateResolution ) // Binlog operation error code list. @@ -362,7 +364,6 @@ const ( codeLoadUnitGenBAList codeLoadTaskWorkerNotMatch codeLoadCheckPointNotMatch - codeLoadBackendNotMatch ) // Sync unit error code. @@ -495,6 +496,7 @@ const ( codeMasterFailToImportFromV10x codeMasterInconsistentOptimistDDLsAndInfo codeMasterOptimisticTableInfobeforeNotExist + codeMasterOptimisticDownstreamMetaNotFound ) // DM-worker error code. @@ -651,6 +653,7 @@ const ( codeSchedulerStartRelayOnBound codeSchedulerStopRelayOnBound codeSchedulerPauseTaskForTransferSource + codeSchedulerWorkerNotFree ) // dmctl error code. @@ -904,6 +907,8 @@ var ( ErrOpenAPITaskConfigExist = New(codeConfigOpenAPITaskConfigExist, ClassConfig, ScopeInternal, LevelLow, "the openapi task config for '%s' already exist", "If you want to override it, please use the overwrite flag.") ErrOpenAPITaskConfigNotExist = New(codeConfigOpenAPITaskConfigNotExist, ClassConfig, ScopeInternal, LevelLow, "the openapi task config for '%s' does not exist", "") ErrConfigCollationCompatibleNotSupport = New(codeCollationCompatibleNotSupport, ClassConfig, ScopeInternal, LevelMedium, "collation compatible %s not supported", "Please check the `collation_compatible` config in task configuration file, which can be set to `loose`/`strict`.") + ErrConfigInvalidLoadMode = New(codeConfigInvalidLoadMode, ClassConfig, ScopeInternal, LevelMedium, "invalid load mode '%s'", "Please choose a valid value in ['sql', 'loader']") + ErrConfigInvalidDuplicateResolution = New(codeConfigInvalidLoadDuplicateResolution, ClassConfig, ScopeInternal, LevelMedium, "invalid load on-duplicate '%s'", "Please choose a valid value in ['replace', 'error', 'ignore']") // Binlog operation error. ErrBinlogExtractPosition = New(codeBinlogExtractPosition, ClassBinlogOp, ScopeInternal, LevelHigh, "", "") @@ -1005,7 +1010,6 @@ var ( ErrLoadUnitGenBAList = New(codeLoadUnitGenBAList, ClassLoadUnit, ScopeInternal, LevelHigh, "generate block allow list", "Please check the `block-allow-list` config in task configuration file.") ErrLoadTaskWorkerNotMatch = New(codeLoadTaskWorkerNotMatch, ClassFunctional, ScopeInternal, LevelHigh, "different worker in load stage, previous worker: %s, current worker: %s", "Please check if the previous worker is online.") ErrLoadTaskCheckPointNotMatch = New(codeLoadCheckPointNotMatch, ClassFunctional, ScopeInternal, LevelHigh, "inconsistent checkpoints between loader and target database", "If you want to redo the whole task, please check that you have not forgotten to add -remove-meta flag for start-task command.") - ErrLoadBackendNotSupport = New(codeLoadBackendNotMatch, ClassFunctional, ScopeInternal, LevelHigh, "DM do not support backend %s ", "If you do not understand the configure `tidb.backend` you can just delete it.") // Sync unit error. ErrSyncerUnitPanic = New(codeSyncerUnitPanic, ClassSyncUnit, ScopeInternal, LevelHigh, "panic error: %v", "") @@ -1140,6 +1144,7 @@ var ( ErrMasterInconsistentOptimisticDDLsAndInfo = New(codeMasterInconsistentOptimistDDLsAndInfo, ClassDMMaster, ScopeInternal, LevelHigh, "inconsistent count of optimistic ddls and table infos, ddls: %d, table info: %d", "") ErrMasterOptimisticTableInfoBeforeNotExist = New(codeMasterOptimisticTableInfobeforeNotExist, ClassDMMaster, ScopeInternal, LevelHigh, "table-info-before not exist in optimistic ddls: %v", "") + ErrMasterOptimisticDownstreamMetaNotFound = New(codeMasterOptimisticDownstreamMetaNotFound, ClassDMMaster, ScopeInternal, LevelHigh, "downstream database config and meta for task %s not found", "") // DM-worker error. ErrWorkerParseFlagSet = New(codeWorkerParseFlagSet, ClassDMWorker, ScopeInternal, LevelMedium, "parse dm-worker config flag set", "") @@ -1301,7 +1306,7 @@ var ( ErrSchedulerStartRelayOnBound = New(codeSchedulerStartRelayOnBound, ClassScheduler, ScopeInternal, LevelLow, "the source has `start-relay` automatically for bound worker, so it can't `start-relay` with worker name now", "Please stop relay by `stop-relay` without worker name first.") ErrSchedulerStopRelayOnBound = New(codeSchedulerStopRelayOnBound, ClassScheduler, ScopeInternal, LevelLow, "the source has `start-relay` automatically for bound worker, so it can't `stop-relay` with worker name now", "Please use `stop-relay` without worker name.") ErrSchedulerPauseTaskForTransferSource = New(codeSchedulerPauseTaskForTransferSource, ClassScheduler, ScopeInternal, LevelLow, "failed to auto pause tasks %s when transfer-source", "Please pause task by `dmctl pause-task`.") - + ErrSchedulerWorkerNotFree = New(codeSchedulerWorkerNotFree, ClassScheduler, ScopeInternal, LevelLow, "dm-worker with name %s not free", "") // dmctl. ErrCtlGRPCCreateConn = New(codeCtlGRPCCreateConn, ClassDMCtl, ScopeInternal, LevelHigh, "can not create grpc connection", "Please check your network connection.") ErrCtlInvalidTLSCfg = New(codeCtlInvalidTLSCfg, ClassDMCtl, ScopeInternal, LevelMedium, "invalid TLS config", "Please check the `ssl-ca`, `ssl-cert` and `ssl-key` config in command line.") diff --git a/dm/relay/meta.go b/dm/relay/meta.go index c48f2c5ae2b..38e50110940 100644 --- a/dm/relay/meta.go +++ b/dm/relay/meta.go @@ -212,7 +212,7 @@ func (lm *LocalMeta) Save(pos mysql.Position, gset gtid.Set) error { lm.BinlogGTID = "" } else { lm.BinlogGTID = gset.String() - lm.gset = gset + lm.gset = gset.Clone() // need to clone and set, in order to avoid the local meta's gset and the input gset referencing the same object, causing contentions later } lm.dirty = true @@ -328,7 +328,7 @@ func (lm *LocalMeta) AddDir(serverUUID string, newPos *mysql.Position, newGTID g } if newGTID != nil { - lm.gset = newGTID + lm.gset = newGTID.Clone() // need to clone and set, in order to avoid the local meta's gset and the input newGTID referencing the same object, causing contentions later lm.BinlogGTID = newGTID.String() } // if newGTID == nil, keep GTID not changed diff --git a/dm/relay/meta_test.go b/dm/relay/meta_test.go index 56c25f69875..558e812b67d 100644 --- a/dm/relay/meta_test.go +++ b/dm/relay/meta_test.go @@ -14,6 +14,7 @@ package relay import ( + "fmt" "os" "path" "strings" @@ -234,3 +235,65 @@ func (r *testMetaSuite) TestLocalMeta(c *C) { currentDir := lm.Dir() c.Assert(strings.HasSuffix(currentDir, cs.uuidWithSuffix), IsTrue) } + +func (r *testMetaSuite) TestLocalMetaPotentialDataRace(c *C) { + var err error + lm := NewLocalMeta("mysql", "/FAKE_DIR") + uuidStr := "85ab69d1-b21f-11e6-9c5e-64006a8978d2" + initGSet, _ := gtid.ParserGTID("mysql", fmt.Sprintf("%s:1", uuidStr)) + lm.(*LocalMeta).currentUUID = uuidStr + err = lm.Save( + mysql.Position{Name: "mysql-bin.000001", Pos: 234}, + initGSet, + ) + c.Assert(err, IsNil) + + ch1 := make(chan error) + ch2 := make(chan error) + pendingCh := make(chan struct{}) + go func() { + <-pendingCh + var err error + defer func() { + ch1 <- err + }() + _, lastGTID := lm.GTID() + var theMGSet mysql.GTIDSet + for i := 2; i < 100; i++ { + theMGSet, err = mysql.ParseGTIDSet("mysql", fmt.Sprintf("%s:1-%d", uuidStr, i*10)) + if err != nil { + return + } + + err = lastGTID.Set(theMGSet) + if err != nil { + return + } + err = lm.Save( + mysql.Position{Name: fmt.Sprintf("mysql-bin.%06d", i), Pos: 123}, + lastGTID, + ) + if err != nil { + return + } + } + }() + var gtidString string + go func() { + <-pendingCh + var err error + defer func() { + ch2 <- err + }() + for i := 0; i < 100; i++ { + _, currentGTID := lm.GTID() + gtidString = currentGTID.String() + } + }() + close(pendingCh) + ch1Err := <-ch1 + ch2Err := <-ch2 + c.Assert(ch1Err, IsNil) + c.Assert(ch2Err, IsNil) + c.Logf("GTID string from the go routine: %s", gtidString) +} diff --git a/dm/syncer/checkpoint.go b/dm/syncer/checkpoint.go index 47c18603fa8..833bde9f798 100644 --- a/dm/syncer/checkpoint.go +++ b/dm/syncer/checkpoint.go @@ -24,12 +24,14 @@ import ( "sync" "time" + "github.com/pingcap/errors" "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/pkg/binlog" "github.com/pingcap/tiflow/dm/pkg/conn" tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/cputil" "github.com/pingcap/tiflow/dm/pkg/dumpling" + fr "github.com/pingcap/tiflow/dm/pkg/func-rollback" "github.com/pingcap/tiflow/dm/pkg/gtid" "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/schema" @@ -60,6 +62,7 @@ var ( globalCpSchema = "" // global checkpoint's cp_schema globalCpTable = "" // global checkpoint's cp_table maxCheckPointTimeout = "1m" + batchFlushPoints = 100 ) type tablePoint struct { @@ -228,7 +231,7 @@ type CheckPoint interface { DeleteSchemaPoint(tctx *tcontext.Context, sourceSchema string) error // IsOlderThanTablePoint checks whether job's checkpoint is older than previous saved checkpoint - IsOlderThanTablePoint(table *filter.Table, point binlog.Location, useLE bool) bool + IsOlderThanTablePoint(table *filter.Table, point binlog.Location, isDDL bool) bool // SaveGlobalPoint saves the global binlog stream's checkpoint // corresponding to Meta.Save @@ -244,8 +247,8 @@ type CheckPoint interface { // corresponding to Meta.Flush FlushPointsExcept(tctx *tcontext.Context, snapshotID int, exceptTables []*filter.Table, extraSQLs []string, extraArgs [][]interface{}) error - // FlushPointWithTableInfo flushed the table point with given table info - FlushPointWithTableInfo(tctx *tcontext.Context, table *filter.Table, ti *model.TableInfo) error + // FlushPointsWithTableInfos flushed the table points with given table infos + FlushPointsWithTableInfos(tctx *tcontext.Context, tables []*filter.Table, tis []*model.TableInfo) error // FlushSafeModeExitPoint flushed the global checkpoint's with given table info FlushSafeModeExitPoint(tctx *tcontext.Context) error @@ -411,17 +414,30 @@ func (cp *RemoteCheckPoint) Snapshot(isSyncFlush bool) *SnapshotInfo { } // Init implements CheckPoint.Init. -func (cp *RemoteCheckPoint) Init(tctx *tcontext.Context) error { +func (cp *RemoteCheckPoint) Init(tctx *tcontext.Context) (err error) { + var db *conn.BaseDB + var dbConns []*dbconn.DBConn + + rollbackHolder := fr.NewRollbackHolder("syncer") + defer func() { + if err != nil { + rollbackHolder.RollbackReverseOrder() + } + }() + checkPointDB := cp.cfg.To checkPointDB.RawDBCfg = config.DefaultRawDBConfig().SetReadTimeout(maxCheckPointTimeout) - db, dbConns, err := dbconn.CreateConns(tctx, cp.cfg, &checkPointDB, 1) + db, dbConns, err = dbconn.CreateConns(tctx, cp.cfg, &checkPointDB, 1) if err != nil { - return err + return } cp.db = db cp.dbConn = dbConns[0] + rollbackHolder.Add(fr.FuncRollback{Name: "CloseRemoteCheckPoint", Fn: cp.Close}) - return cp.prepare(tctx) + err = cp.prepare(tctx) + + return } // Close implements CheckPoint.Close. @@ -561,14 +577,12 @@ func (cp *RemoteCheckPoint) DeleteSchemaPoint(tctx *tcontext.Context, sourceSche } // IsOlderThanTablePoint implements CheckPoint.IsOlderThanTablePoint. -// For GTID replication, go-mysql will only update GTID set in a XID event after the rows event, for example, the binlog events are: -// - Query event e1, location is gset1 -// - Rows event e2, location is gset1 -// - XID event, location is gset2 -// We should note that e1 is not older than e2 -// For binlog position replication, currently DM will split rows changes of an event to jobs, so some job may has save position. -// if useLE is true, we use less than or equal. -func (cp *RemoteCheckPoint) IsOlderThanTablePoint(table *filter.Table, location binlog.Location, useLE bool) bool { +// This function is used to skip old binlog events. Table checkpoint is saved after dispatching a binlog event. +// - For GTID based and position based replication, DML handling is different. When using position based, each event has +// unique position so we have confident to skip event which is <= table checkpoint. When using GTID based, there may +// be more than one event with same GTID, so we can only skip event which is < table checkpoint. +// - DDL will not have unique position or GTID, so we can always skip events <= table checkpoint. +func (cp *RemoteCheckPoint) IsOlderThanTablePoint(table *filter.Table, location binlog.Location, isDDL bool) bool { cp.RLock() defer cp.RUnlock() sourceSchema, sourceTable := table.Schema, table.Name @@ -583,7 +597,7 @@ func (cp *RemoteCheckPoint) IsOlderThanTablePoint(table *filter.Table, location oldLocation := point.MySQLLocation() cp.logCtx.L().Debug("compare table location whether is newer", zap.Stringer("location", location), zap.Stringer("old location", oldLocation)) - if useLE { + if isDDL || !cp.cfg.EnableGTID { return binlog.CompareLocation(location, oldLocation, cp.cfg.EnableGTID) <= 0 } return binlog.CompareLocation(location, oldLocation, cp.cfg.EnableGTID) < 0 @@ -688,7 +702,9 @@ func (cp *RemoteCheckPoint) FlushPointsExcept( if snapshotCp.globalPoint != nil { cp.globalPoint.flushBy(*snapshotCp.globalPoint) + cp.Lock() cp.globalPointSaveTime = snapshotCp.globalPointSaveTime + cp.Unlock() } for _, point := range points { @@ -698,45 +714,63 @@ func (cp *RemoteCheckPoint) FlushPointsExcept( return nil } -// FlushPointWithTableInfo implements CheckPoint.FlushPointWithTableInfo. -func (cp *RemoteCheckPoint) FlushPointWithTableInfo(tctx *tcontext.Context, table *filter.Table, ti *model.TableInfo) error { +// FlushPointsWithTableInfos implements CheckPoint.FlushPointsWithTableInfos. +func (cp *RemoteCheckPoint) FlushPointsWithTableInfos(tctx *tcontext.Context, tables []*filter.Table, tis []*model.TableInfo) error { cp.Lock() defer cp.Unlock() - sourceSchema, sourceTable := table.Schema, table.Name - sqls := make([]string, 0, 1) - args := make([][]interface{}, 0, 10) - point := newBinlogPoint(binlog.NewLocation(cp.cfg.Flavor), binlog.NewLocation(cp.cfg.Flavor), nil, nil, cp.cfg.EnableGTID) - - if tablePoints, ok := cp.points[sourceSchema]; ok { - if p, ok2 := tablePoints[sourceTable]; ok2 { - point = p - } - } - - tiBytes, err := json.Marshal(ti) - if err != nil { - return terror.ErrSchemaTrackerCannotSerialize.Delegate(err, sourceSchema, sourceTable) + // should not happened + if len(tables) != len(tis) { + return errors.Errorf("the length of the tables is not equal to the length of the table infos, left: %d, right: %d", len(tables), len(tis)) } - location := point.MySQLLocation() - sql2, arg := cp.genUpdateSQL(sourceSchema, sourceTable, location, nil, tiBytes, false) - sqls = append(sqls, sql2) - args = append(args, arg) + for i := 0; i < len(tables); i += batchFlushPoints { + end := i + batchFlushPoints + if end > len(tables) { + end = len(tables) + } - // use a new context apart from syncer, to make sure when syncer call `cancel` checkpoint could update - tctx2, cancel := tctx.WithContext(context.Background()).WithTimeout(utils.DefaultDBTimeout) - defer cancel() - _, err = cp.dbConn.ExecuteSQL(tctx2, sqls, args...) - if err != nil { - return err - } + sqls := make([]string, 0, batchFlushPoints) + args := make([][]interface{}, 0, batchFlushPoints) + points := make([]*binlogPoint, 0, batchFlushPoints) + for j := i; j < end; j++ { + table := tables[j] + ti := tis[j] + sourceSchema, sourceTable := table.Schema, table.Name + + var point *binlogPoint + // if point already in memory, use it + if tablePoints, ok := cp.points[sourceSchema]; ok { + if p, ok2 := tablePoints[sourceTable]; ok2 { + point = p + } + } + // create new point + if point == nil { + cp.saveTablePoint(table, cp.globalPoint.MySQLLocation(), ti) + point = cp.points[sourceSchema][sourceTable] + } + tiBytes, err := json.Marshal(ti) + if err != nil { + return terror.ErrSchemaTrackerCannotSerialize.Delegate(err, sourceSchema, sourceTable) + } + location := point.MySQLLocation() + sql, arg := cp.genUpdateSQL(sourceSchema, sourceTable, location, nil, tiBytes, false) + sqls = append(sqls, sql) + args = append(args, arg) + points = append(points, point) + } + // use a new context apart from syncer, to make sure when syncer call `cancel` checkpoint could update + tctx2, cancel := tctx.WithContext(context.Background()).WithTimeout(utils.DefaultDBTimeout) + defer cancel() + _, err := cp.dbConn.ExecuteSQL(tctx2, sqls, args...) + if err != nil { + return err + } - err = point.save(point.savedPoint.location, ti) - if err != nil { - return err + for _, point := range points { + point.flush() + } } - point.flush() - return nil } @@ -830,6 +864,7 @@ func (cp *RemoteCheckPoint) Rollback(schemaTracker *schema.Tracker) { cp.RLock() defer cp.RUnlock() cp.globalPoint.rollback(schemaTracker, "") + tablesToCreate := make(map[string]map[string]*model.TableInfo) for schemaName, mSchema := range cp.points { for tableName, point := range mSchema { table := &filter.Table{ @@ -850,13 +885,18 @@ func (cp *RemoteCheckPoint) Rollback(schemaTracker *schema.Tracker) { if err := schemaTracker.CreateSchemaIfNotExists(schemaName); err != nil { logger.Error("failed to rollback schema on schema tracker: cannot create schema", log.ShortError(err)) } - if err := schemaTracker.CreateTableIfNotExists(table, point.savedPoint.ti); err != nil { - logger.Error("failed to rollback schema on schema tracker: cannot create table", log.ShortError(err)) + if _, ok := tablesToCreate[schemaName]; !ok { + tablesToCreate[schemaName] = map[string]*model.TableInfo{} } + tablesToCreate[schemaName][tableName] = point.savedPoint.ti } } } } + logger := cp.logCtx.L().WithFields(zap.Reflect("batch create table", tablesToCreate)) + if err := schemaTracker.BatchCreateTableIfNotExist(tablesToCreate); err != nil { + logger.Error("failed to rollback schema on schema tracker: cannot create table", log.ShortError(err)) + } // drop any tables in the tracker if no corresponding checkpoint exists. for _, schema := range schemaTracker.AllSchemas() { diff --git a/dm/syncer/dml_worker.go b/dm/syncer/dml_worker.go index 051bb140f05..78c4e48dd1f 100644 --- a/dm/syncer/dml_worker.go +++ b/dm/syncer/dml_worker.go @@ -196,17 +196,28 @@ func (w *DMLWorker) executeJobs(queueID int, jobCh chan *job) { // executeBatchJobs execute jobs with batch size. func (w *DMLWorker) executeBatchJobs(queueID int, jobs []*job) { var ( - affect int - db = w.toDBConns[queueID] - err error - dmls = make([]*DML, 0, len(jobs)) + affect int + queries []string + args [][]interface{} + db = w.toDBConns[queueID] + err error + dmls = make([]*DML, 0, len(jobs)) ) defer func() { if err == nil { w.successFunc(queueID, len(dmls), jobs) } else { - w.fatalFunc(jobs[affect], err) + if len(queries) == len(jobs) { + w.fatalFunc(jobs[affect], err) + } else { + w.logger.Warn("length of queries not equals length of jobs, cannot determine which job failed", zap.Int("queries", len(queries)), zap.Int("jobs", len(jobs))) + newJob := job{ + startLocation: jobs[0].startLocation, + currentLocation: jobs[len(jobs)-1].currentLocation, + } + w.fatalFunc(&newJob, err) + } } }() @@ -230,7 +241,7 @@ func (w *DMLWorker) executeBatchJobs(queueID int, jobs []*job) { for _, j := range jobs { dmls = append(dmls, j.dml) } - queries, args := w.genSQLs(dmls) + queries, args = w.genSQLs(dmls) failpoint.Inject("WaitUserCancel", func(v failpoint.Value) { t := v.(int) time.Sleep(time.Duration(t) * time.Second) @@ -245,6 +256,13 @@ func (w *DMLWorker) executeBatchJobs(queueID int, jobs []*job) { affect, err = 0, terror.ErrDBExecuteFailed.Delegate(errors.New("SafeModeExit"), "mock") } }) + + failpoint.Inject("ErrorOnLastDML", func(_ failpoint.Value) { + if len(queries) > len(jobs) { + w.logger.Error("error on last queries", zap.Int("queries", len(queries)), zap.Int("jobs", len(jobs))) + affect, err = len(queries)-1, terror.ErrDBExecuteFailed.Delegate(errors.New("ErrorOnLastDML"), "mock") + } + }) } // genSQLs generate SQLs in single row mode or multiple rows mode. diff --git a/dm/syncer/optimist.go b/dm/syncer/optimist.go index aa6a1b9b0db..575dcdb5473 100644 --- a/dm/syncer/optimist.go +++ b/dm/syncer/optimist.go @@ -191,7 +191,7 @@ func (s *Syncer) handleQueryEventOptimistic(qec *queryEventContext) error { qec.shardingDDLInfo = trackInfos[0] job := newDDLJob(qec) - err = s.handleJobFunc(job) + _, err = s.handleJobFunc(job) if err != nil { return err } @@ -218,18 +218,3 @@ func (s *Syncer) handleQueryEventOptimistic(qec *queryEventContext) error { s.tctx.L().Info("finish to handle ddls in optimistic shard mode", zap.String("event", "query"), zap.Stringer("queryEventContext", qec)) return nil } - -// trackInitTableInfoOptimistic tries to get the initial table info (before modified by other tables) and track it in optimistic shard mode. -func (s *Syncer) trackInitTableInfoOptimistic(sourceTable, targetTable *filter.Table) (*model.TableInfo, error) { - ti, err := s.optimist.GetTableInfo(targetTable.Schema, targetTable.Name) - if err != nil { - return nil, terror.ErrSchemaTrackerCannotGetTable.Delegate(err, sourceTable) - } - if ti != nil { - err = s.schemaTracker.CreateTableIfNotExists(sourceTable, ti) - if err != nil { - return nil, terror.ErrSchemaTrackerCannotCreateTable.Delegate(err, sourceTable) - } - } - return ti, nil -} diff --git a/dm/syncer/schema.go b/dm/syncer/schema.go index 979d814783e..eca48fb41d0 100644 --- a/dm/syncer/schema.go +++ b/dm/syncer/schema.go @@ -138,7 +138,7 @@ func (s *Syncer) OperateSchema(ctx context.Context, req *pb.OperateWorkerSchemaR if req.Flush { log.L().Info("flush table info", zap.String("table info", newSQL)) - err = s.checkpoint.FlushPointWithTableInfo(tcontext.NewContext(ctx, log.L()), sourceTable, ti) + err = s.checkpoint.FlushPointsWithTableInfos(tcontext.NewContext(ctx, log.L()), []*filter.Table{sourceTable}, []*model.TableInfo{ti}) if err != nil { return "", err } diff --git a/dm/syncer/shardddl/optimist.go b/dm/syncer/shardddl/optimist.go index 027626d3726..4b076e96008 100644 --- a/dm/syncer/shardddl/optimist.go +++ b/dm/syncer/shardddl/optimist.go @@ -17,6 +17,7 @@ import ( "context" "sync" + filter "github.com/pingcap/tidb-tools/pkg/table-filter" "github.com/pingcap/tidb/parser/model" "go.etcd.io/etcd/clientv3" "go.uber.org/zap" @@ -70,6 +71,25 @@ func (o *Optimist) Init(sourceTables map[string]map[string]map[string]map[string return err } +// Tables clone and return tables +// first one is sourceTable, second one is targetTable. +func (o *Optimist) Tables() [][]filter.Table { + o.mu.Lock() + defer o.mu.Unlock() + + tbls := make([][]filter.Table, 0) + for downSchema, downTables := range o.tables.Tables { + for downTable, upSchemas := range downTables { + for upSchema, upTables := range upSchemas { + for upTable := range upTables { + tbls = append(tbls, []filter.Table{{Schema: upSchema, Name: upTable}, {Schema: downSchema, Name: downTable}}) + } + } + } + } + return tbls +} + // Reset resets the internal state of the optimist. func (o *Optimist) Reset() { o.mu.Lock() @@ -161,24 +181,6 @@ func (o *Optimist) DoneOperation(op optimism.Operation) error { return nil } -// GetTableInfo tries to get the init schema of the downstream table. -func (o *Optimist) GetTableInfo(downSchema, downTable string) (*model.TableInfo, error) { - if downTable == "" { - return nil, nil - } - - is, rev, err := optimism.GetInitSchema(o.cli, o.task, downSchema, downTable) - if err != nil { - return nil, err - } - if is.IsEmpty() { - o.logger.Info("no init schema exists", zap.String("schema", downSchema), zap.String("table", downTable), zap.Int64("revision", rev)) - } else { - o.logger.Info("got init schema", zap.Stringer("init schema", is)) - } - return is.TableInfo, nil -} - // PendingInfo returns the shard DDL info which is pending to handle. func (o *Optimist) PendingInfo() *optimism.Info { o.mu.RLock() diff --git a/dm/syncer/shardddl/optimist_test.go b/dm/syncer/shardddl/optimist_test.go index fa616f51bae..7b2a76bfbed 100644 --- a/dm/syncer/shardddl/optimist_test.go +++ b/dm/syncer/shardddl/optimist_test.go @@ -36,7 +36,7 @@ var _ = Suite(&testOptimist{}) // clear keys in etcd test cluster. func clearOptimistTestSourceInfoOperation(c *C) { - c.Assert(optimism.ClearTestInfoOperationSchema(etcdTestCli), IsNil) + c.Assert(optimism.ClearTestInfoOperationColumn(etcdTestCli), IsNil) } func createTableInfo(c *C, p *parser.Parser, se sessionctx.Context, tableID int64, sql string) *model.TableInfo { @@ -95,6 +95,9 @@ func (t *testOptimist) TestOptimist(c *C) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() + tables := o.Tables() + c.Assert(len(tables), Equals, 0) + // init with some source tables. err := o.Init(sourceTables) c.Assert(err, IsNil) @@ -104,6 +107,9 @@ func (t *testOptimist) TestOptimist(c *C) { c.Assert(stm[task], HasLen, 1) c.Assert(stm[task][source], DeepEquals, o.tables) + tables = o.Tables() + c.Assert(len(tables), Equals, 4) + // no info and operation in pending. c.Assert(o.PendingInfo(), IsNil) c.Assert(o.PendingOperation(), IsNil) @@ -206,42 +212,3 @@ func (t *testOptimist) TestOptimist(c *C) { c.Assert(o.PendingInfo(), IsNil) c.Assert(o.PendingOperation(), IsNil) } - -func (t *testOptimist) TestGetTableInfo(c *C) { - defer clearOptimistTestSourceInfoOperation(c) - - var ( - task = "test-get-table-info" - source = "mysql-replica-1" - logger = log.L() - o = NewOptimist(&logger, etcdTestCli, task, source) - - downSchema = "foo" - downTable = "bar" - p = parser.New() - se = mock.NewContext() - tblID int64 = 111 - is = optimism.NewInitSchema(task, downSchema, downTable, - createTableInfo(c, p, se, tblID, `CREATE TABLE bar (id INT PRIMARY KEY)`)) - ) - - // no table info exist now - ti, err := o.GetTableInfo(downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(ti, IsNil) - - // put the table schema. - _, putted, err := optimism.PutInitSchemaIfNotExist(etcdTestCli, is) - c.Assert(err, IsNil) - c.Assert(putted, IsTrue) - - // can get the table info now. - ti, err = o.GetTableInfo(downSchema, downTable) - c.Assert(err, IsNil) - c.Assert(ti, DeepEquals, is.TableInfo) - - // no table info for database. - ti, err = o.GetTableInfo(downSchema, "") - c.Assert(err, IsNil) - c.Assert(ti, IsNil) -} diff --git a/dm/syncer/sharding_group.go b/dm/syncer/sharding_group.go index d330642a9f2..2ff48dba4ff 100644 --- a/dm/syncer/sharding_group.go +++ b/dm/syncer/sharding_group.go @@ -80,6 +80,7 @@ import ( "github.com/pingcap/tiflow/dm/pkg/conn" tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/cputil" + fr "github.com/pingcap/tiflow/dm/pkg/func-rollback" "github.com/pingcap/tiflow/dm/pkg/terror" "github.com/pingcap/tiflow/dm/pkg/utils" "github.com/pingcap/tiflow/dm/syncer/dbconn" @@ -427,17 +428,31 @@ func (k *ShardingGroupKeeper) AddGroup(targetTable *filter.Table, sourceIDs []st } // Init does initialization staff. -func (k *ShardingGroupKeeper) Init() error { +func (k *ShardingGroupKeeper) Init() (err error) { + var db *conn.BaseDB + var dbConns []*dbconn.DBConn + + rollbackHolder := fr.NewRollbackHolder("syncer") + defer func() { + if err != nil { + rollbackHolder.RollbackReverseOrder() + } + }() + k.clear() sgkDB := k.cfg.To sgkDB.RawDBCfg = config.DefaultRawDBConfig().SetReadTimeout(maxCheckPointTimeout) - db, dbConns, err := dbconn.CreateConns(k.tctx, k.cfg, &sgkDB, 1) + db, dbConns, err = dbconn.CreateConns(k.tctx, k.cfg, &sgkDB, 1) if err != nil { - return err + return } k.db = db k.dbConn = dbConns[0] - return k.prepare() + rollbackHolder.Add(fr.FuncRollback{Name: "CloseShardingGroupKeeper", Fn: k.Close}) + + err = k.prepare() + + return } // clear clears all sharding groups. diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 7906719a19b..212a3fa9bb1 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -219,7 +219,7 @@ type Syncer struct { isQueryEvent bool } - handleJobFunc func(*job) error + handleJobFunc func(*job) (bool, error) flushSeq int64 // `lower_case_table_names` setting of upstream db @@ -735,14 +735,6 @@ func (s *Syncer) getTableInfo(tctx *tcontext.Context, sourceTable, targetTable * return ti, nil } - // in optimistic shard mode, we should try to get the init schema (the one before modified by other tables) first. - if s.cfg.ShardMode == config.ShardOptimistic { - ti, err = s.trackInitTableInfoOptimistic(sourceTable, targetTable) - if err != nil { - return nil, err - } - } - // if the table does not exist (IsTableNotExists(err)), continue to fetch the table from downstream and create it. if ti == nil { err = s.trackTableInfoFromDownstream(tctx, sourceTable, targetTable) @@ -991,7 +983,7 @@ func (s *Syncer) checkShouldFlush() error { // TODO: move to syncer/job.go // handleJob will do many actions based on job type. -func (s *Syncer) handleJob(job *job) (err error) { +func (s *Syncer) handleJob(job *job) (added2Queue bool, err error) { skipCheckFlush := false defer func() { if !skipCheckFlush && err == nil { @@ -1012,7 +1004,7 @@ func (s *Syncer) handleJob(job *job) (err error) { if waitXIDStatus(s.waitXIDJob.Load()) == waitComplete && job.tp != flush { s.tctx.L().Info("All jobs is completed before syncer close, the coming job will be reject", zap.Any("job", job)) - return nil + return } switch job.tp { @@ -1020,15 +1012,16 @@ func (s *Syncer) handleJob(job *job) (err error) { s.waitXIDJob.CAS(int64(waiting), int64(waitComplete)) s.saveGlobalPoint(job.location) s.isTransactionEnd = true - return nil + return case skip: s.updateReplicationJobTS(job, skipJobIdx) - return nil + return } // 2. send the job to queue s.addJob(job) + added2Queue = true // 3. after job is sent to queue @@ -1038,14 +1031,14 @@ func (s *Syncer) handleJob(job *job) (err error) { // caller s.isTransactionEnd = false skipCheckFlush = true - return nil + return case ddl: s.jobWg.Wait() // skip rest logic when downstream error if s.execError.Load() != nil { // nolint:nilerr - return nil + return } s.updateReplicationJobTS(job, ddlJobIdx) @@ -1058,7 +1051,7 @@ func (s *Syncer) handleJob(job *job) (err error) { failpoint.Inject("FlushCheckpointStage", func(val failpoint.Value) { err = handleFlushCheckpointStage(3, val.(int), "before save checkpoint") if err != nil { - failpoint.Return(err) + failpoint.Return() } }) // save global checkpoint for DDL @@ -1078,19 +1071,22 @@ func (s *Syncer) handleJob(job *job) (err error) { failpoint.Inject("FlushCheckpointStage", func(val failpoint.Value) { err = handleFlushCheckpointStage(4, val.(int), "before flush checkpoint") if err != nil { - failpoint.Return(err) + failpoint.Return() } }) skipCheckFlush = true - return s.flushCheckPoints() + err = s.flushCheckPoints() + return case flush: s.jobWg.Wait() skipCheckFlush = true - return s.flushCheckPoints() + err = s.flushCheckPoints() + return case asyncFlush: skipCheckFlush = true } - return err + // nolint:nakedret + return } func (s *Syncer) saveGlobalPoint(globalLocation binlog.Location) { @@ -1222,7 +1218,7 @@ func (s *Syncer) afterFlushCheckpoint(task *checkpointFlushTask) error { s.tctx.L().Info("after async flushed checkpoint, gc stale causality keys", zap.Int64("flush job seq", task.asyncflushJob.flushSeq)) s.addJob(newGCJob(task.asyncflushJob.flushSeq)) } else { - s.tctx.L().Info("after async flushed checkpoint, gc all causality keys") + s.tctx.L().Info("after sync flushed checkpoint, gc all causality keys") s.addJob(newGCJob(math.MaxInt64)) } @@ -1488,7 +1484,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { if err != nil { return err } - if s.cfg.Mode == config.ModeAll && fresh { + if fresh && s.cfg.Mode == config.ModeAll { delLoadTask = true flushCheckpoint = true err = s.loadTableStructureFromDump(ctx) @@ -1496,7 +1492,12 @@ func (s *Syncer) Run(ctx context.Context) (err error) { tctx.L().Warn("error happened when load table structure from dump files", zap.Error(err)) cleanDumpFile = false } - } else { + if s.cfg.ShardMode == config.ShardOptimistic { + s.flushOptimisticTableInfos(tctx) + } + } + + if s.cfg.Mode == config.ModeIncrement || !fresh { cleanDumpFile = false } @@ -2053,7 +2054,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { } job := newXIDJob(currentLocation, startLocation, currentLocation) - err2 = s.handleJobFunc(job) + _, err2 = s.handleJobFunc(job) case *replication.GenericEvent: if e.Header.EventType == replication.HEARTBEAT_EVENT { // flush checkpoint even if there are no real binlog events @@ -2325,9 +2326,9 @@ func (s *Syncer) handleRowsEvent(ev *replication.RowsEvent, ec eventContext) err startTime := time.Now() for i := range dmls { job := newDMLJob(jobType, sourceTable, targetTable, dmls[i], &ec) - err = s.handleJobFunc(job) - if err != nil { - return err + added2Queue, err2 := s.handleJobFunc(job) + if err2 != nil || !added2Queue { + return err2 } } metrics.DispatchBinlogDurationHistogram.WithLabelValues(jobType.String(), s.cfg.Name, s.cfg.SourceID).Observe(time.Since(startTime).Seconds()) @@ -2692,7 +2693,7 @@ func (s *Syncer) handleQueryEventNoSharding(qec *queryEventContext) error { }) job := newDDLJob(qec) - err := s.handleJobFunc(job) + _, err := s.handleJobFunc(job) if err != nil { return err } @@ -2914,7 +2915,7 @@ func (s *Syncer) handleQueryEventPessimistic(qec *queryEventContext) error { }) job := newDDLJob(qec) - err = s.handleJobFunc(job) + _, err = s.handleJobFunc(job) if err != nil { return err } @@ -3168,7 +3169,7 @@ func (s *Syncer) loadTableStructureFromDump(ctx context.Context) error { continue } } - logger.Info("fetch table structure form dump files", + logger.Info("fetch table structure from dump files", zap.Strings("database", dbs), zap.Any("tables", tables)) for _, db := range dbs { @@ -3299,7 +3300,8 @@ func (s *Syncer) closeDBs() { // make newJob's sql argument empty to distinguish normal sql and skips sql. func (s *Syncer) recordSkipSQLsLocation(ec *eventContext) error { job := newSkipJob(ec) - return s.handleJobFunc(job) + _, err := s.handleJobFunc(job) + return err } // flushJobs add a flush job and wait for all jobs finished. @@ -3308,7 +3310,8 @@ func (s *Syncer) flushJobs() error { flushJobSeq := s.getFlushSeq() s.tctx.L().Info("flush all jobs", zap.Stringer("global checkpoint", s.checkpoint), zap.Int64("flush job seq", flushJobSeq)) job := newFlushJob(s.cfg.WorkerCount, flushJobSeq) - return s.handleJobFunc(job) + _, err := s.handleJobFunc(job) + return err } func (s *Syncer) reSyncBinlog(tctx tcontext.Context, location binlog.Location) error { @@ -3758,3 +3761,23 @@ func calculateChanSize(queueSize, workerCount int, compact bool) int { } return chanSize } + +func (s *Syncer) flushOptimisticTableInfos(tctx *tcontext.Context) { + tbls := s.optimist.Tables() + sourceTables := make([]*filter.Table, 0, len(tbls)) + tableInfos := make([]*model.TableInfo, 0, len(tbls)) + for _, tbl := range tbls { + sourceTable := tbl[0] + targetTable := tbl[1] + tableInfo, err := s.getTableInfo(tctx, &sourceTable, &targetTable) + if err != nil { + tctx.L().Error("failed to get table infos", log.ShortError(err)) + continue + } + sourceTables = append(sourceTables, &sourceTable) + tableInfos = append(tableInfos, tableInfo) + } + if err := s.checkpoint.FlushPointsWithTableInfos(tctx, sourceTables, tableInfos); err != nil { + tctx.L().Error("failed to flush table points with table infos", log.ShortError(err)) + } +} diff --git a/dm/syncer/syncer_test.go b/dm/syncer/syncer_test.go index c4347e18ea6..1eac0b668fa 100644 --- a/dm/syncer/syncer_test.go +++ b/dm/syncer/syncer_test.go @@ -1472,7 +1472,7 @@ func (s *Syncer) mockFinishJob(jobs []*expectJob) { } } -func (s *Syncer) addJobToMemory(job *job) error { +func (s *Syncer) addJobToMemory(job *job) (bool, error) { log.L().Info("add job to memory", zap.Stringer("job", job)) switch job.tp { @@ -1512,7 +1512,7 @@ func (s *Syncer) addJobToMemory(job *job) error { } } - return nil + return true, nil } func (s *Syncer) setupMockCheckpoint(c *C, checkPointDBConn *sql.Conn, checkPointMock sqlmock.Sqlmock) { diff --git a/dm/tests/_utils/env_variables b/dm/tests/_utils/env_variables index 2818e72294c..a0ae74c9ef1 100755 --- a/dm/tests/_utils/env_variables +++ b/dm/tests/_utils/env_variables @@ -27,6 +27,8 @@ WORKER2_PORT=8263 WORKER3_PORT=8264 WORKER4_PORT=18262 WORKER5_PORT=18263 +WORKER1_NAME=worker1 +WORKER2_NAME=worker2 SOURCE_ID1="mysql-replica-01" SOURCE_ID2="mysql-replica-02" RESET_MASTER=${RESET_MASTER:-true} diff --git a/dm/tests/_utils/run_dm_ctl_with_tls_and_retry b/dm/tests/_utils/run_dm_ctl_with_tls_and_retry index cdc049defce..5f7ed3e9df0 100755 --- a/dm/tests/_utils/run_dm_ctl_with_tls_and_retry +++ b/dm/tests/_utils/run_dm_ctl_with_tls_and_retry @@ -39,6 +39,9 @@ for ((k = 0; k < 10; k++)); do break fi done + if [ $# -eq 0 ]; then + cat $dmctl_log + fi if $all_matched; then exit 0 fi diff --git a/dm/tests/_utils/shardddl_lib.sh b/dm/tests/_utils/shardddl_lib.sh index 3860a971204..b5f1ffc9c30 100644 --- a/dm/tests/_utils/shardddl_lib.sh +++ b/dm/tests/_utils/shardddl_lib.sh @@ -54,3 +54,39 @@ function restart_master() { run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT } + +function restart_worker1() { + echo "restart dm-worker1" + ps aux | grep worker1 | awk '{print $2}' | xargs kill || true + check_port_offline $WORKER1_PORT 20 + 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 +} + +function restart_worker2() { + echo "restart dm-worker2" + ps aux | grep worker2 | awk '{print $2}' | xargs kill || true + check_port_offline $WORKER2_PORT 20 + 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 +} + +function restart_task() { + echo "restart task" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-task test" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $1" + + if [[ "$task_conf" == *"single"* ]]; then + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"unit\": \"Sync\"" 1 + elif [[ "$task_conf" == *"double"* ]]; then + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"unit\": \"Sync\"" 2 + fi +} diff --git a/dm/tests/_utils/test_prepare b/dm/tests/_utils/test_prepare index f268325da4e..b4ea29bf170 100644 --- a/dm/tests/_utils/test_prepare +++ b/dm/tests/_utils/test_prepare @@ -324,7 +324,7 @@ function init_cluster(){ } function get_master_status() { - arr=$(echo "show master status;" | MYSQL_PWD=123456 mysql -uroot -h127.0.0.1 -P3306 | awk 'NR==2') + arr=$(echo "show master status;" | MYSQL_PWD=123456 mysql -uroot -h$1 -P$2 | awk 'NR==2') echo $arr } diff --git a/dm/tests/adjust_gtid/conf/diff_config_revert_1.toml b/dm/tests/adjust_gtid/conf/diff_config_revert_1.toml deleted file mode 100644 index 455ed32451f..00000000000 --- a/dm/tests/adjust_gtid/conf/diff_config_revert_1.toml +++ /dev/null @@ -1,31 +0,0 @@ -# diff Configuration. - -check-thread-count = 4 - -export-fix-sql = true - -check-struct-only = false - - -[task] - output-dir = "/tmp/ticdc_dm_test/output" - - source-instances = ["tidb0"] - - target-instance = "mysql1" - - target-check-tables = ["adjust_gtid.t?*"] - - -[data-sources] -[data-sources.mysql1] -host = "127.0.0.1" -port = 3306 -user = "root" -password = "123456" - -[data-sources.tidb0] -host = "127.0.0.1" -port = 4000 -user = "test" -password = "123456" diff --git a/dm/tests/adjust_gtid/conf/diff_config_revert_2.toml b/dm/tests/adjust_gtid/conf/diff_config_revert_2.toml deleted file mode 100644 index 6d38ad7fb17..00000000000 --- a/dm/tests/adjust_gtid/conf/diff_config_revert_2.toml +++ /dev/null @@ -1,31 +0,0 @@ -# diff Configuration. - -check-thread-count = 4 - -export-fix-sql = true - -check-struct-only = false - - -[task] - output-dir = "/tmp/ticdc_dm_test/output" - - source-instances = ["tidb0"] - - target-instance = "mysql2" - - target-check-tables = ["adjust_gtid.t?*"] - - -[data-sources] -[data-sources.mysql2] -host = "127.0.0.1" -port = 3307 -user = "root" -password = "123456" - -[data-sources.tidb0] -host = "127.0.0.1" -port = 4000 -user = "test" -password = "123456" diff --git a/dm/tests/adjust_gtid/run.sh b/dm/tests/adjust_gtid/run.sh index 107485d1e31..01676329be1 100755 --- a/dm/tests/adjust_gtid/run.sh +++ b/dm/tests/adjust_gtid/run.sh @@ -85,8 +85,7 @@ function run() { "start-task $WORK_DIR/dm-task.yaml --remove-meta" # use sync_diff_inspector to check full dump loader - check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_1.toml - check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_2.toml + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml name1=$(grep "Log: " $WORK_DIR/worker1/dumped_data.$TASK_NAME/metadata | awk -F: '{print $2}' | tr -d ' ') pos1=$(grep "Pos: " $WORK_DIR/worker1/dumped_data.$TASK_NAME/metadata | awk -F: '{print $2}' | tr -d ' ') @@ -128,8 +127,7 @@ function run() { check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT # use sync_diff_inspector to check incremental dump loader - check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_1.toml - check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_2.toml + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml run_sql_both_source "SET @@GLOBAL.SQL_MODE='ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_ENGINE_SUBSTITUTION'" run_sql_both_source "SET @@global.time_zone = 'SYSTEM';" diff --git a/dm/tests/all_mode/conf/dm-task.yaml b/dm/tests/all_mode/conf/dm-task.yaml index 17553c1b0f7..851519b4760 100644 --- a/dm/tests/all_mode/conf/dm-task.yaml +++ b/dm/tests/all_mode/conf/dm-task.yaml @@ -56,6 +56,7 @@ loaders: global: pool-size: 16 dir: "./dumped_data" + import-mode: sql syncers: global: diff --git a/dm/tests/all_mode/run.sh b/dm/tests/all_mode/run.sh index 043511d9f43..d933b763d1b 100755 --- a/dm/tests/all_mode/run.sh +++ b/dm/tests/all_mode/run.sh @@ -176,7 +176,9 @@ function test_stop_task_before_checkpoint() { dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 # generate uncomplete checkpoint - dmctl_start_task "$cur/conf/dm-task.yaml" "--remove-meta" + cp $cur/conf/dm-task.yaml $WORK_DIR/dm-task.yaml + sed -i "s/import-mode: sql/import-mode: loader/" $WORK_DIR/dm-task.yaml + dmctl_start_task "$WORK_DIR/dm-task.yaml" "--remove-meta" check_log_contain_with_retry 'wait loader stop after init checkpoint' $WORK_DIR/worker1/log/dm-worker.log check_log_contain_with_retry 'wait loader stop after init checkpoint' $WORK_DIR/worker2/log/dm-worker.log run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ @@ -195,14 +197,14 @@ function test_stop_task_before_checkpoint() { check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT # stop-task before load checkpoint - dmctl_start_task "$cur/conf/dm-task.yaml" + dmctl_start_task $WORK_DIR/dm-task.yaml check_log_contain_with_retry 'wait loader stop before load checkpoint' $WORK_DIR/worker1/log/dm-worker.log check_log_contain_with_retry 'wait loader stop before load checkpoint' $WORK_DIR/worker2/log/dm-worker.log run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "stop-task test" \ "\"result\": true" 3 - dmctl_start_task "$cur/conf/dm-task.yaml" + dmctl_start_task $WORK_DIR/dm-task.yaml check_sync_diff $WORK_DIR $cur/conf/diff_config.toml run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "stop-task test" \ diff --git a/dm/tests/dmctl_basic/check_list/check_task.sh b/dm/tests/dmctl_basic/check_list/check_task.sh index 08016f50796..d3a628c95af 100644 --- a/dm/tests/dmctl_basic/check_list/check_task.sh +++ b/dm/tests/dmctl_basic/check_list/check_task.sh @@ -35,6 +35,13 @@ function check_task_error_database_config() { "Please check the database connection and the database config in configuration file" 1 } +function check_task_wrong_start_time_format() { + task_conf=$1 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "check-task $task_conf --start-time '20060102 150405'" \ + "error while parse start-time" 1 +} + function check_task_error_count() { task_conf=$1 # 10 errors diff --git a/dm/tests/dmctl_basic/check_list/operate_source.sh b/dm/tests/dmctl_basic/check_list/operate_source.sh index 58e82590db3..5c02f31443c 100644 --- a/dm/tests/dmctl_basic/check_list/operate_source.sh +++ b/dm/tests/dmctl_basic/check_list/operate_source.sh @@ -3,7 +3,7 @@ function operate_source_empty_arg() { run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "operate-source" \ - "operate-source \[config-file ...\] \[--print-sample-config\] \[flags\]" 1 + "operate-source \[config-file ...\] \[-w worker\] \[--print-sample-config\] \[flags\]" 1 } function operate_source_wrong_config_file() { diff --git a/dm/tests/dmctl_basic/check_list/start_task.sh b/dm/tests/dmctl_basic/check_list/start_task.sh index acbb79cdf3c..6e41a0b7b70 100644 --- a/dm/tests/dmctl_basic/check_list/start_task.sh +++ b/dm/tests/dmctl_basic/check_list/start_task.sh @@ -11,3 +11,10 @@ function start_task_wrong_config_file() { "start-task not_exists_config_file" \ "error in get file content" 1 } + +function start_task_wrong_start_time_format() { + task_conf=$1 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $task_conf --start-time '20060102 150405'" \ + "error while parse start-time" 1 +} diff --git a/dm/tests/dmctl_basic/conf/get_task.yaml b/dm/tests/dmctl_basic/conf/get_task.yaml index 4cc9cfac480..30aa6e566a5 100644 --- a/dm/tests/dmctl_basic/conf/get_task.yaml +++ b/dm/tests/dmctl_basic/conf/get_task.yaml @@ -142,6 +142,8 @@ loaders: load-01: pool-size: 16 dir: ./dumped_data + import-mode: sql + on-duplicate: replace syncers: sync-01: meta-file: "" @@ -160,6 +162,5 @@ syncers: clean-dump-file: true ansi-quotes: false remove-meta: false -tidb: null experimental: async-checkpoint-flush: false diff --git a/dm/tests/dmctl_basic/run.sh b/dm/tests/dmctl_basic/run.sh index a1d82ee7dea..858713ee737 100755 --- a/dm/tests/dmctl_basic/run.sh +++ b/dm/tests/dmctl_basic/run.sh @@ -279,6 +279,7 @@ function run() { echo "dmctl_check_task" check_task_pass $TASK_CONF + check_task_wrong_start_time_format $cur/conf/dm-task3.yaml check_task_not_pass $cur/conf/dm-task2.yaml check_task_error_count $cur/conf/dm-task3.yaml @@ -296,6 +297,7 @@ function run() { check_task_error_database_config $WORK_DIR/dm-task-error-database-config.yaml echo "dmctl_start_task" + start_task_wrong_start_time_format $cur/conf/dm-task3.yaml dmctl_start_task check_sync_diff $WORK_DIR $cur/conf/diff_config.toml run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ diff --git a/dm/tests/dmctl_command/conf/dm-task.yaml b/dm/tests/dmctl_command/conf/dm-task.yaml index 702e88919e3..21b6540b32b 100644 --- a/dm/tests/dmctl_command/conf/dm-task.yaml +++ b/dm/tests/dmctl_command/conf/dm-task.yaml @@ -7,7 +7,6 @@ meta-schema: "dm_meta" heartbeat-update-interval: 1 heartbeat-report-interval: 1 remove-meta: true -tidb: null target-database: host: "127.0.0.1" diff --git a/dm/tests/dmctl_command/run.sh b/dm/tests/dmctl_command/run.sh index 8d9ba8efbd4..2b73bb8ac55 100644 --- a/dm/tests/dmctl_command/run.sh +++ b/dm/tests/dmctl_command/run.sh @@ -81,7 +81,22 @@ function run() { cp $cur/conf/source1.yaml $WORK_DIR/source1.yaml cp $cur/conf/source2.yaml $WORK_DIR/source2.yaml sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker1/relay_log" $WORK_DIR/source1.yaml - dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-source create $WORK_DIR/source1.yaml -w $WORKER1_NAME" \ + "\"result\": true" 2 \ + "\"source\": \"mysql-replica-01\"" 1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "list-member --name worker1" \ + "\"stage\": \"bound\"" 1 \ + "\"source\": \"mysql-replica-01\"" 1 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-source create $WORK_DIR/source2.yaml -w wrong-worker" \ + "\"result\": false" 1 \ + "not exists" 1 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-source create $WORK_DIR/source2.yaml -w worker1" \ + "\"result\": false" 1 \ + "not free" 1 dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 # check wrong do-tables diff --git a/dm/tests/downstream_more_column/run.sh b/dm/tests/downstream_more_column/run.sh index eb8eca00da0..edfef1dabe1 100755 --- a/dm/tests/downstream_more_column/run.sh +++ b/dm/tests/downstream_more_column/run.sh @@ -43,7 +43,7 @@ function run() { # start DM task in incremental mode # schemaTracker create table from downstream - master_status=($(get_master_status)) + master_status=($(get_master_status $MYSQL_HOST1 $MYSQL_PORT1)) cp $cur/conf/dm-task-incremental.yaml $WORK_DIR/dm-task-incremental.yaml sed -i "s/binlog-gtid-placeholder/${master_status[2]}/g" $WORK_DIR/dm-task-incremental.yaml run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ diff --git a/dm/tests/full_mode/conf/diff_config_revert_1.toml b/dm/tests/full_mode/conf/diff_config_revert_1.toml deleted file mode 100644 index f690ce6e804..00000000000 --- a/dm/tests/full_mode/conf/diff_config_revert_1.toml +++ /dev/null @@ -1,30 +0,0 @@ -# diff Configuration. - -check-thread-count = 4 - -export-fix-sql = true - -check-struct-only = false - -[task] - output-dir = "/tmp/ticdc_dm_test/output" - - source-instances = ["tidb0"] - - target-instance = "mysql1" - - target-check-tables = ["full_mode.t?*"] - - -[data-sources] -[data-sources.mysql1] -host = "127.0.0.1" -port = 3306 -user = "root" -password = "123456" - -[data-sources.tidb0] -host = "127.0.0.1" -port = 4000 -user = "test" -password = "123456" diff --git a/dm/tests/full_mode/conf/diff_config_revert_2.toml b/dm/tests/full_mode/conf/diff_config_revert_2.toml deleted file mode 100644 index 84a3ecd2d43..00000000000 --- a/dm/tests/full_mode/conf/diff_config_revert_2.toml +++ /dev/null @@ -1,30 +0,0 @@ -# diff Configuration. - -check-thread-count = 4 - -export-fix-sql = true - -check-struct-only = false - -[task] - output-dir = "/tmp/ticdc_dm_test/output" - - source-instances = ["tidb0"] - - target-instance = "mysql2" - - target-check-tables = ["full_mode.t?*"] - - -[data-sources] -[data-sources.mysql2] -host = "127.0.0.1" -port = 3307 -user = "root" -password = "123456" - -[data-sources.tidb0] -host = "127.0.0.1" -port = 4000 -user = "test" -password = "123456" diff --git a/dm/tests/full_mode/run.sh b/dm/tests/full_mode/run.sh index e26387ba331..80fd8975c52 100755 --- a/dm/tests/full_mode/run.sh +++ b/dm/tests/full_mode/run.sh @@ -56,7 +56,6 @@ function fail_acquire_global_lock() { "you need (at least one of) the RELOAD privilege(s) for this operation" 2 cleanup_data full_mode - cleanup_data_upstream full_mode cleanup_process $* } @@ -101,6 +100,7 @@ function escape_schema() { # start DM task only dmctl_start_task "$WORK_DIR/dm-task.yaml" "--remove-meta" + check_sync_diff $WORK_DIR $WORK_DIR/diff_config.toml check_log_contain_with_retry 'clean dump files' $WORK_DIR/worker1/log/dm-worker.log check_log_contain_with_retry 'clean dump files' $WORK_DIR/worker2/log/dm-worker.log @@ -114,7 +114,6 @@ function escape_schema() { check_metric $WORKER2_PORT 'dumpling_dump_finished_tables' 3 0 3 cleanup_data full/mode - cleanup_data_upstream full/mode cleanup_process $* } @@ -127,8 +126,7 @@ function empty_data() { init_cluster dmctl_start_task "$cur/conf/dm-task.yaml" "--remove-meta" - check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_1.toml - check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_2.toml + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml sleep 1 run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ @@ -141,7 +139,6 @@ function empty_data() { check_log_contains $WORK_DIR/worker2/log/dm-worker.log "progress=\"100.00 %\"" cleanup_data full_mode - cleanup_data_upstream full_mode cleanup_process $* } @@ -190,8 +187,7 @@ function run() { dmctl_start_task "$cur/conf/dm-task.yaml" "--remove-meta" # use sync_diff_inspector to check full dump loader - check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_1.toml - check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_2.toml + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml echo "check dump files have been cleaned" ls $WORK_DIR/worker1/dumped_data.test && exit 1 || echo "worker1 auto removed dump files" diff --git a/dm/tests/import_goroutine_leak/conf/dm-task.yaml b/dm/tests/import_goroutine_leak/conf/dm-task.yaml index 210bca31e2f..d10c5591722 100644 --- a/dm/tests/import_goroutine_leak/conf/dm-task.yaml +++ b/dm/tests/import_goroutine_leak/conf/dm-task.yaml @@ -34,6 +34,7 @@ loaders: global: pool-size: 16 dir: "./dumped_data" + import-mode: "loader" syncers: global: diff --git a/dm/tests/import_v10x/conf/task.yaml b/dm/tests/import_v10x/conf/task.yaml index c5d449f1810..a8756979ed2 100644 --- a/dm/tests/import_v10x/conf/task.yaml +++ b/dm/tests/import_v10x/conf/task.yaml @@ -87,6 +87,8 @@ loaders: load-01: pool-size: 16 dir: ./dumped_data + import-mode: sql + on-duplicate: replace syncers: sync-01: meta-file: "" @@ -119,6 +121,5 @@ syncers: clean-dump-file: false ansi-quotes: false remove-meta: false -tidb: null experimental: async-checkpoint-flush: false diff --git a/dm/tests/incremental_mode/conf/diff_config_revert_1.toml b/dm/tests/incremental_mode/conf/diff_config_revert_1.toml deleted file mode 100644 index 45328c12217..00000000000 --- a/dm/tests/incremental_mode/conf/diff_config_revert_1.toml +++ /dev/null @@ -1,30 +0,0 @@ -# diff Configuration. - -check-thread-count = 4 - -export-fix-sql = true - -check-struct-only = false - -[task] - output-dir = "/tmp/ticdc_dm_test/output" - - source-instances = ["tidb0"] - - target-instance = "mysql1" - - target-check-tables = ["incremental_mode.t?*"] - - -[data-sources] -[data-sources.mysql1] -host = "127.0.0.1" -port = 3306 -user = "root" -password = "123456" - -[data-sources.tidb0] -host = "127.0.0.1" -port = 4000 -user = "test" -password = "123456" diff --git a/dm/tests/incremental_mode/conf/diff_config_revert_2.toml b/dm/tests/incremental_mode/conf/diff_config_revert_2.toml deleted file mode 100644 index 16ed402a93f..00000000000 --- a/dm/tests/incremental_mode/conf/diff_config_revert_2.toml +++ /dev/null @@ -1,30 +0,0 @@ -# diff Configuration. - -check-thread-count = 4 - -export-fix-sql = true - -check-struct-only = false - -[task] - output-dir = "/tmp/ticdc_dm_test/output" - - source-instances = ["tidb0"] - - target-instance = "mysql2" - - target-check-tables = ["incremental_mode.t?*"] - - -[data-sources] -[data-sources.mysql2] -host = "127.0.0.1" -port = 3307 -user = "root" -password = "123456" - -[data-sources.tidb0] -host = "127.0.0.1" -port = 4000 -user = "test" -password = "123456" diff --git a/dm/tests/incremental_mode/run.sh b/dm/tests/incremental_mode/run.sh index 011c72cefd7..e86ace8800b 100755 --- a/dm/tests/incremental_mode/run.sh +++ b/dm/tests/incremental_mode/run.sh @@ -76,8 +76,7 @@ function run() { sed -i "s/binlog-pos-placeholder-2/4/g" $WORK_DIR/dm-task.yaml dmctl_start_task $WORK_DIR/dm-task.yaml - check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_1.toml - check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_2.toml + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml dmctl_stop_task $TASK_NAME @@ -266,8 +265,7 @@ function run() { "resume-task test" \ "\"result\": true" 3 - check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_1.toml - check_sync_diff $WORK_DIR $cur/conf/diff_config_revert_2.toml + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml # test rotate binlog, after rotate and ddl, master binlog should be equal to sync binlog run_sql "flush logs;" $MYSQL_PORT1 $MYSQL_PASSWORD1 diff --git a/dm/tests/lightning_load_task/conf/dm-task.yaml b/dm/tests/lightning_load_task/conf/dm-task.yaml index b29aa864d12..a8f2e93d964 100644 --- a/dm/tests/lightning_load_task/conf/dm-task.yaml +++ b/dm/tests/lightning_load_task/conf/dm-task.yaml @@ -25,9 +25,6 @@ mysql-instances: loader-config-name: "global" syncer-config-name: "global" -tidb: - backend: "tidb" - block-allow-list: instance: do-dbs: ["load_task1"] @@ -43,6 +40,7 @@ loaders: global: pool-size: 16 dir: "./dumped_data" + import-mode: "sql" syncers: global: diff --git a/dm/tests/lightning_load_task/conf/dm-task2.yaml b/dm/tests/lightning_load_task/conf/dm-task2.yaml index a751da96d85..14c8b07645d 100644 --- a/dm/tests/lightning_load_task/conf/dm-task2.yaml +++ b/dm/tests/lightning_load_task/conf/dm-task2.yaml @@ -25,9 +25,6 @@ mysql-instances: loader-config-name: "global" syncer-config-name: "global" -tidb: - backend: "tidb" - block-allow-list: instance: do-dbs: ["load_task2"] diff --git a/dm/tests/lightning_load_task/conf/dm-task3.yaml b/dm/tests/lightning_load_task/conf/dm-task3.yaml index e6395139e8e..0a06a4eabfb 100644 --- a/dm/tests/lightning_load_task/conf/dm-task3.yaml +++ b/dm/tests/lightning_load_task/conf/dm-task3.yaml @@ -23,9 +23,6 @@ block-allow-list: instance: do-dbs: ["load_task3"] -tidb: - backend: "tidb" - mydumpers: global: threads: 4 diff --git a/dm/tests/lightning_load_task/conf/dm-task4.yaml b/dm/tests/lightning_load_task/conf/dm-task4.yaml index d2abd5092f3..7e153eb4c49 100644 --- a/dm/tests/lightning_load_task/conf/dm-task4.yaml +++ b/dm/tests/lightning_load_task/conf/dm-task4.yaml @@ -30,9 +30,6 @@ mydumpers: skip-tz-utc: true extra-args: "" -tidb: - backend: "tidb" - loaders: global: pool-size: 16 diff --git a/dm/tests/lightning_load_task/run.sh b/dm/tests/lightning_load_task/run.sh index 6848fc344da..6e130f06623 100755 --- a/dm/tests/lightning_load_task/run.sh +++ b/dm/tests/lightning_load_task/run.sh @@ -33,7 +33,7 @@ function test_worker_restart() { "Please check if the previous worker is online." 1 # worker1 online - export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LightningLoadDataSlowDownByTask=return(\"load_task1\")" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LoadDataSlowDownByTask=return(\"load_task1\")" run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $CONF_DIR/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT @@ -82,7 +82,7 @@ function test_transfer_two_sources() { "\"unit\": \"Load\"" 1 # worker2 online - export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LightningLoadDataSlowDown=sleep(15000)" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LoadDataSlowDown=sleep(15000)" run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $CONF_DIR/dm-worker2.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT @@ -184,7 +184,7 @@ function run() { check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT1 # worker1 loading load_task1 - export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LightningLoadDataSlowDownByTask=return(\"load_task1\")" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LoadDataSlowDownByTask=return(\"load_task1\")" run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $CONF_DIR/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT cp $CONF_DIR/source1.yaml $WORK_DIR/source1.yaml @@ -192,7 +192,7 @@ function run() { dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 # worker2 loading load_task2 - export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LightningLoadDataSlowDownByTask=return(\"load_task2\")" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LoadDataSlowDownByTask=return(\"load_task2\")" run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $CONF_DIR/dm-worker2.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT cp $CONF_DIR/source2.yaml $WORK_DIR/source2.yaml @@ -200,7 +200,7 @@ function run() { dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 # worker3 loading load_task3 - export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LightningLoadDataSlowDownByTask=return(\"load_task3\")" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LoadDataSlowDownByTask=return(\"load_task3\")" run_dm_worker $WORK_DIR/worker3 $WORKER3_PORT $CONF_DIR/dm-worker3.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER3_PORT @@ -208,8 +208,8 @@ function run() { dmctl_start_task "$cur/conf/dm-task.yaml" "--remove-meta" dmctl_start_task "$cur/conf/dm-task2.yaml" "--remove-meta" - check_log_contain_with_retry 'inject failpoint LightningLoadDataSlowDownByTask' $WORK_DIR/worker1/log/dm-worker.log - check_log_contain_with_retry 'inject failpoint LightningLoadDataSlowDownByTask' $WORK_DIR/worker2/log/dm-worker.log + check_log_contain_with_retry 'inject failpoint LoadDataSlowDownByTask in lightning loader' $WORK_DIR/worker1/log/dm-worker.log + check_log_contain_with_retry 'inject failpoint LoadDataSlowDownByTask in lightning loader' $WORK_DIR/worker2/log/dm-worker.log run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status load_task1" \ "\"unit\": \"Load\"" 1 \ diff --git a/dm/tests/lightning_mode/conf/dm-task.yaml b/dm/tests/lightning_mode/conf/dm-task.yaml index 2b3d43261c0..d33b6ed3157 100644 --- a/dm/tests/lightning_mode/conf/dm-task.yaml +++ b/dm/tests/lightning_mode/conf/dm-task.yaml @@ -37,9 +37,6 @@ black-white-list: # compatible with deprecated config do-dbs: ["lightning_mode"] ignore-dbs: ["ignore_db"] -tidb: - backend: "tidb" - filters: # filter rules, mysql instance can ref rules in it user-filter-1: schema-pattern: "*" # pattern of the upstream schema name, wildcard characters (*?) are supported @@ -58,6 +55,7 @@ loaders: global: pool-size: 16 dir: "./dumped_data" + import-mode: "sql" syncers: global: diff --git a/dm/tests/load_interrupt/conf/dm-task.yaml b/dm/tests/load_interrupt/conf/dm-task.yaml index d09b9d04347..833ba15b113 100644 --- a/dm/tests/load_interrupt/conf/dm-task.yaml +++ b/dm/tests/load_interrupt/conf/dm-task.yaml @@ -34,6 +34,7 @@ loaders: global: pool-size: 16 dir: "./dumped_data" + import-mode: "loader" syncers: global: diff --git a/dm/tests/new_relay/configs/tasks/test.yaml b/dm/tests/new_relay/configs/tasks/test.yaml index 8c174e41f7b..3040c12592c 100644 --- a/dm/tests/new_relay/configs/tasks/test.yaml +++ b/dm/tests/new_relay/configs/tasks/test.yaml @@ -43,6 +43,8 @@ loaders: load-01: pool-size: 16 dir: ./dumped_data + import-mode: sql + on-duplicate: replace syncers: sync-01: meta-file: "" diff --git a/dm/tests/others_integration_2.txt b/dm/tests/others_integration_2.txt index e6f69950689..676318c3ce2 100644 --- a/dm/tests/others_integration_2.txt +++ b/dm/tests/others_integration_2.txt @@ -10,4 +10,5 @@ openapi duplicate_event tracker_ignored_ddl extend_column +shardddl_optimistic gbk diff --git a/dm/tests/print_status/run.sh b/dm/tests/print_status/run.sh index 67469c5ecf4..7f11f1b3d66 100755 --- a/dm/tests/print_status/run.sh +++ b/dm/tests/print_status/run.sh @@ -64,21 +64,9 @@ function check_print_status() { echo "checking print status" # check load unit print status status_file=$WORK_DIR/worker1/log/loader_status.log - grep -oP "\[unit=load\] \[finished_bytes=[0-9]+\] \[total_bytes=59637\] \[total_file_count=3\] \[progress=.*\]" $WORK_DIR/worker1/log/dm-worker.log >$status_file - #grep -oP "loader.*\Kfinished_bytes = [0-9]+, total_bytes = [0-9]+, total_file_count = [0-9]+, progress = .*" $WORK_DIR/worker1/log/dm-worker.log > $status_file + grep -oP "\[unit=lightning-load\] \[IsCanceled=false\] \[finished_bytes=59637\] \[total_bytes=59637\] \[progress=.*\]" $WORK_DIR/worker1/log/dm-worker.log >$status_file status_count=$(wc -l $status_file | awk '{print $1}') - [ $status_count -ge 2 ] - count=0 - cat $status_file | while read -r line; do - total_file_count=$(echo "$line" | awk '{print $(NF-2)}' | tr -d "[total_file_count=" | tr -d "]") - [ $total_file_count -eq 3 ] - count=$((count + 1)) - if [ $count -eq $status_count ]; then - finished_bytes=$(echo "$line" | awk '{print $2}' | tr -d "[finished_bytes=" | tr -d "]") - total_bytes=$(echo "$line" | awk '{print $3}' | tr -d "[total_file_count" | tr -d "]") - [[ "$finished_bytes" -eq "$total_bytes" ]] - fi - done + [ $status_count -eq 1 ] echo "check load unit print status success" # check sync unit print status diff --git a/dm/tests/sequence_sharding_optimistic/run.sh b/dm/tests/sequence_sharding_optimistic/run.sh index 58ea6fd456c..20f5f5a0d38 100755 --- a/dm/tests/sequence_sharding_optimistic/run.sh +++ b/dm/tests/sequence_sharding_optimistic/run.sh @@ -64,9 +64,9 @@ run() { "query-status $task_name" \ "\"stage\": \"Paused\"" 2 - # try to get schema for the table, but can't get because no DDL/DML replicated yet. + # try to get schema for the table, table exists for optimistic. curl -X PUT ${API_URL} -d '{"op":1, "task":"sequence_sharding_optimistic", "sources": ["mysql-replica-01"], "database":"sharding_seq_opt", "table":"t1"}' >${WORK_DIR}/get_schema.log - check_log_contains ${WORK_DIR}/get_schema.log "Table 'sharding_seq_opt.t1' doesn't exist" 1 + check_log_contains ${WORK_DIR}/get_schema.log 'CREATE TABLE `t1` ( `id` bigint(20) NOT NULL, `c1` varchar(20) DEFAULT NULL, `c2` varchar(20) DEFAULT NULL, PRIMARY KEY (`id`) .*) ENGINE=InnoDB DEFAULT CHARSET=latin1 COLLATE=latin1_bin' 1 # resume task manually. run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ diff --git a/dm/tests/shardddl1/run.sh b/dm/tests/shardddl1/run.sh index 4acfef21348..101b9d4e7f8 100644 --- a/dm/tests/shardddl1/run.sh +++ b/dm/tests/shardddl1/run.sh @@ -394,10 +394,17 @@ function DM_UpdateBARule_CASE() { sed -i 's/do-dbs: \["shardddl1","shardddl2"\]/do-dbs: \["shardddl1"\]/g' $WORK_DIR/task.yaml echo 'ignore-checking-items: ["schema_of_shard_tables"]' >>$WORK_DIR/task.yaml + # source1: db1.tb1(id,new_col1,new_col3) + # source2: db1.tb1(id) run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "start-task $WORK_DIR/task.yaml" \ "\"result\": true" 3 + # no lock exist when task begin + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "show-ddl-locks" \ + "no DDL lock exists" 1 + run_sql_source1 "insert into ${shardddl1}.${tb1} values(13,13,13);" run_sql_source2 "insert into ${shardddl1}.${tb1} values(14);" run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb};" "count(1): 14" @@ -406,7 +413,7 @@ function DM_UpdateBARule_CASE() { run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "show-ddl-locks" \ - "\"ID\": \"test-\`shardddl\`.\`tb\`\"" 1 + "no DDL lock exists" 1 run_sql_source1 "alter table ${shardddl1}.${tb1} drop column new_col1" run_sql_source2 "alter table ${shardddl1}.${tb1} add column new_col3 int" @@ -783,6 +790,32 @@ function DM_CAUSALITY_USE_DOWNSTREAM_SCHEMA() { "clean_table" "" } +function DM_DML_EXECUTE_ERROR_CASE() { + run_sql_source1 "insert into ${shardddl1}.${tb1}(a,b) values(1,1)" + run_sql_source1 "update ${shardddl1}.${tb1} set b=b+1 where a=1" + + check_log_contain_with_retry "length of queries not equals length of jobs" $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "\"RawCause\": \"ErrorOnLastDML\"" 1 \ + "Paused" 1 +} + +function DM_DML_EXECUTE_ERROR() { + ps aux | grep dm-worker | awk '{print $2}' | xargs kill || true + check_port_offline $WORKER1_PORT 20 + check_port_offline $WORKER2_PORT 20 + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/syncer/ErrorOnLastDML=return()' + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml + 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:$WORKER1_PORT + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT + + run_case DML_EXECUTE_ERROR "single-source-no-sharding" \ + "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key, b int);\"" \ + "clean_table" "" +} + function run() { init_cluster init_database @@ -799,6 +832,7 @@ function run() { DM_RestartMaster DM_ADD_DROP_COLUMNS DM_COLUMN_INDEX + DM_DML_EXECUTE_ERROR start=1 end=5 for i in $(seq -f "%03g" ${start} ${end}); do diff --git a/dm/tests/shardddl_optimistic/conf/diff_config.toml b/dm/tests/shardddl_optimistic/conf/diff_config.toml new file mode 100644 index 00000000000..cf3abc02891 --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/diff_config.toml @@ -0,0 +1,44 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] + output-dir = "/tmp/ticdc_dm_test/output" + + source-instances = ["mysql1", "mysql2"] + + target-instance = "tidb0" + + target-check-tables = ["shardddl.tb"] + +[routes.rule1] +schema-pattern = "shardddl[1-2]" +table-pattern = "tb*" +target-schema = "shardddl" +target-table = "tb" + + +[data-sources] +[data-sources.mysql1] +host = "127.0.0.1" +port = 3306 +user = "root" +password = "123456" +route-rules = ["rule1"] + +[data-sources.mysql2] +host = "127.0.0.1" +port = 3307 +user = "root" +password = "123456" +route-rules = ["rule1"] + +[data-sources.tidb0] +host = "127.0.0.1" +port = 4000 +user = "test" +password = "123456" diff --git a/dm/tests/shardddl_optimistic/conf/dm-master.toml b/dm/tests/shardddl_optimistic/conf/dm-master.toml new file mode 100644 index 00000000000..458b3e124f0 --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/dm-master.toml @@ -0,0 +1,10 @@ +# Master Configuration. +name = "master1" +master-addr = ":8261" +advertise-addr = "127.0.0.1:8261" +peer-urls = "127.0.0.1:8291" +initial-cluster = "master1=http://127.0.0.1:8291" +auto-compaction-retention = "3s" + +[experimental] +openapi = true diff --git a/dm/tests/shardddl_optimistic/conf/dm-worker1.toml b/dm/tests/shardddl_optimistic/conf/dm-worker1.toml new file mode 100644 index 00000000000..6f1d1b5344f --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/dm-worker1.toml @@ -0,0 +1,2 @@ +name = "worker1" +join = "127.0.0.1:8261" \ No newline at end of file diff --git a/dm/tests/shardddl_optimistic/conf/dm-worker2.toml b/dm/tests/shardddl_optimistic/conf/dm-worker2.toml new file mode 100644 index 00000000000..83949162686 --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/dm-worker2.toml @@ -0,0 +1,2 @@ +name = "worker2" +join = "127.0.0.1:8261" \ No newline at end of file diff --git a/dm/tests/shardddl_optimistic/conf/double-source-optimistic-incr.yaml b/dm/tests/shardddl_optimistic/conf/double-source-optimistic-incr.yaml new file mode 100644 index 00000000000..541c103ddbc --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/double-source-optimistic-incr.yaml @@ -0,0 +1,63 @@ +--- +name: test +task-mode: incremental +is-sharding: true +shard-mode: "optimistic" +meta-schema: "dm_meta" +ignore-checking-items: ["all"] + +target-database: + host: "127.0.0.1" + port: 4000 + user: "test" + password: "/Q7B9DizNLLTTfiZHv9WoEAKamfpIUs=" + +mysql-instances: + - source-id: "mysql-replica-01" + block-allow-list: "instance" + route-rules: ["sharding-table-rules","sharding-schema-rules"] + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + meta: + binlog-pos: pos-holder + binlog-name: name-holder + - source-id: "mysql-replica-02" + block-allow-list: "instance" + route-rules: ["sharding-table-rules","sharding-schema-rules"] + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + meta: + binlog-gtid: gtid-holder + +block-allow-list: + instance: + do-dbs: ["shardddl1","shardddl2"] + +routes: + sharding-table-rules: + schema-pattern: "shardddl*" + target-schema: "shardddl" + table-pattern: "tb*" + target-table: "tb" + sharding-schema-rules: + schema-pattern: "shardddl*" + target-schema: "shardddl" + +mydumpers: + global: + threads: 4 + chunk-filesize: 64 + skip-tz-utc: true + extra-args: "" + +loaders: + global: + pool-size: 16 + dir: "./dumped_data" + +syncers: + global: + worker-count: 16 + batch: 100 diff --git a/dm/tests/shardddl_optimistic/conf/double-source-optimistic.yaml b/dm/tests/shardddl_optimistic/conf/double-source-optimistic.yaml new file mode 100644 index 00000000000..8e60eab721d --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/double-source-optimistic.yaml @@ -0,0 +1,58 @@ +--- +name: test +task-mode: all +is-sharding: true +shard-mode: "optimistic" +meta-schema: "dm_meta" +ignore-checking-items: ["all"] + +target-database: + host: "127.0.0.1" + port: 4000 + user: "test" + password: "/Q7B9DizNLLTTfiZHv9WoEAKamfpIUs=" + +mysql-instances: + - source-id: "mysql-replica-01" + block-allow-list: "instance" + route-rules: ["sharding-table-rules","sharding-schema-rules"] + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + - source-id: "mysql-replica-02" + block-allow-list: "instance" + route-rules: ["sharding-table-rules","sharding-schema-rules"] + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + +block-allow-list: + instance: + do-dbs: ["shardddl1","shardddl2"] + +routes: + sharding-table-rules: + schema-pattern: "shardddl*" + target-schema: "shardddl" + table-pattern: "tb*" + target-table: "tb" + sharding-schema-rules: + schema-pattern: "shardddl*" + target-schema: "shardddl" + +mydumpers: + global: + threads: 4 + chunk-filesize: 64 + skip-tz-utc: true + extra-args: "" + +loaders: + global: + pool-size: 16 + dir: "./dumped_data" + +syncers: + global: + worker-count: 16 + batch: 100 diff --git a/dm/tests/shardddl_optimistic/conf/source1.yaml b/dm/tests/shardddl_optimistic/conf/source1.yaml new file mode 100644 index 00000000000..175e07df7a5 --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/source1.yaml @@ -0,0 +1,11 @@ +source-id: mysql-replica-01 +flavor: '' +enable-gtid: false +enable-relay: false +from: + host: 127.0.0.1 + user: root + password: /Q7B9DizNLLTTfiZHv9WoEAKamfpIUs= + port: 3306 +checker: + check-enable: false diff --git a/dm/tests/shardddl_optimistic/conf/source2.yaml b/dm/tests/shardddl_optimistic/conf/source2.yaml new file mode 100644 index 00000000000..e6508fd9d4f --- /dev/null +++ b/dm/tests/shardddl_optimistic/conf/source2.yaml @@ -0,0 +1,11 @@ +source-id: mysql-replica-02 +flavor: '' +enable-gtid: true +enable-relay: true +from: + host: 127.0.0.1 + user: root + password: /Q7B9DizNLLTTfiZHv9WoEAKamfpIUs= + port: 3307 +checker: + check-enable: false diff --git a/dm/tests/shardddl_optimistic/run.sh b/dm/tests/shardddl_optimistic/run.sh new file mode 100644 index 00000000000..09acecfa9bb --- /dev/null +++ b/dm/tests/shardddl_optimistic/run.sh @@ -0,0 +1,426 @@ +#!/bin/bash + +set -eu + +cur=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +source $cur/../_utils/test_prepare +WORK_DIR=$TEST_DIR/$TEST_NAME +source $cur/../_utils/shardddl_lib.sh + +function DM_DIFFERENT_SCHEMA_FULL_CASE() { + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 4" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(5);" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(6,'6');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(7,'77');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(8,'8','88');" + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column c text;" + # source1.tb1(a,c); source1.tb2(a,b); source2.tb1(a,c); source2.tb2(a,b,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(9,'999');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(10,'1010');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(11,'111111');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(12,'1212','121212');" + + run_sql_source2 "alter table ${shardddl1}.${tb2} drop column b;" + # source1.tb1(a,c); source1.tb2(a,b); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(13,'131313');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(14,'1414');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(15,'151515');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(16,'161616');" + + run_sql_source1 "alter table ${shardddl1}.${tb2} drop column b;" + # source1.tb1(a,c); source1.tb2(a); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(17,'171717');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(18);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(19,'191919');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(20,'202020');" + + run_sql_source1 "alter table ${shardddl1}.${tb2} add column c text;" + # source1.tb1(a,c); source1.tb2(a,c); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(21,'212121');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(22,'222222');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(23,'232323');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(24,'242424');" + + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 24" + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml +} + +function DM_DIFFERENT_SCHEMA_FULL() { + # create table with different schema, init data, and create table in downstream manually + run_case DIFFERENT_SCHEMA_FULL "double-source-optimistic" \ + "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key);\"; \ + run_sql_source1 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10));\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb1} (a int primary key, c text);\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10), c text);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb1} values(1);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb2} values(2,'22');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb1} values(3,'333');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb2} values(4,'44','444');\"; \ + run_sql_tidb \"create database if not exists ${shardddl};\"; \ + run_sql_tidb \"create table ${shardddl}.${tb} (a int primary key, b varchar(10), c text);\"" \ + "clean_table" "optimistic" +} + +function DM_DIFFERENT_SCHEMA_INCREMENTAL_CASE() { + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 4" + + # get checkpoint + source1_status=($(get_master_status $MYSQL_HOST1 $MYSQL_PORT1)) + source2_status=($(get_master_status $MYSQL_HOST2 $MYSQL_PORT2)) + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "pause-task test" \ + "\"result\": true" 3 + + # save schema + curl -X GET http://127.0.0.1:8261/api/v1/tasks/test/sources/mysql-replica-01/schemas/${shardddl1}/${tb1} | jq -r .schema_create_sql >$WORK_DIR/schema11.sql + curl -X GET http://127.0.0.1:8261/api/v1/tasks/test/sources/mysql-replica-01/schemas/${shardddl1}/${tb2} | jq -r .schema_create_sql >$WORK_DIR/schema12.sql + curl -X GET http://127.0.0.1:8261/api/v1/tasks/test/sources/mysql-replica-02/schemas/${shardddl1}/${tb1} | jq -r .schema_create_sql >$WORK_DIR/schema21.sql + curl -X GET http://127.0.0.1:8261/api/v1/tasks/test/sources/mysql-replica-02/schemas/${shardddl1}/${tb2} | jq -r .schema_create_sql >$WORK_DIR/schema22.sql + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-task test" \ + "\"result\": true" 3 + + # incremental data + run_sql_source1 "insert into ${shardddl1}.${tb1} values(5);" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(6,'6');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(7,'77');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(8,'8','88');" + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column c text;" + # source1.tb1(a,c); source1.tb2(a,b); source2.tb1(a,c); source2.tb2(a,b,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(9,'999');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(10,'1010');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(11,'111111');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(12,'1212','121212');" + + run_sql_source2 "alter table ${shardddl1}.${tb2} drop column b;" + # source1.tb1(a,c); source1.tb2(a,b); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(13,'131313');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(14,'1414');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(15,'151515');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(16,'161616');" + + run_sql_source1 "alter table ${shardddl1}.${tb2} drop column b;" + # source1.tb1(a,c); source1.tb2(a); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(17,'171717');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(18);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(19,'191919');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(20,'202020');" + + run_sql_source1 "alter table ${shardddl1}.${tb2} add column c text;" + # source1.tb1(a,c); source1.tb2(a,c); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(21,'212121');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(22,'222222');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(23,'232323');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(24,'242424');" + + # start task with current checkpoint + sed "s/pos-holder/${source1_status[1]}/g" $cur/conf/double-source-optimistic-incr.yaml >$WORK_DIR/task.yaml + sed -i "s/name-holder/${source1_status[0]}/g" $WORK_DIR/task.yaml + sed -i "s/gtid-holder/${source2_status[2]}/g" $WORK_DIR/task.yaml + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $WORK_DIR/task.yaml --remove-meta" + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "Column count doesn't match" 2 + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-schema set -s mysql-replica-01 test -d ${shardddl1} -t ${tb1} $WORK_DIR/schema11.sql" \ + "\"result\": true" 2 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-schema set -s mysql-replica-01 test -d ${shardddl1} -t ${tb2} $WORK_DIR/schema12.sql" \ + "\"result\": true" 2 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-schema set -s mysql-replica-02 test -d ${shardddl1} -t ${tb1} $WORK_DIR/schema21.sql" \ + "\"result\": true" 2 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "operate-schema set -s mysql-replica-02 test -d ${shardddl1} -t ${tb2} $WORK_DIR/schema22.sql" \ + "\"result\": true" 2 + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "resume-task test" \ + "\"result\": true" 3 + + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 24" + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml +} + +function DM_DIFFERENT_SCHEMA_INCREMENTAL() { + # create table with different schema, init data, and create table in downstream manually + run_case DIFFERENT_SCHEMA_INCREMENTAL "double-source-optimistic" \ + "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key);\"; \ + run_sql_source1 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10));\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb1} (a int primary key, c text);\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10), c text);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb1} values(1);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb2} values(2,'22');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb1} values(3,'333');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb2} values(4,'44','444');\"; \ + run_sql_tidb \"create database if not exists ${shardddl};\"; \ + run_sql_tidb \"create table ${shardddl}.${tb} (a int primary key, b varchar(10), c text);\"" \ + "clean_table" "optimistic" +} + +function DM_RESTART_TASK_MASTER_WORKER_CASE() { + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 4" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(5);" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(6,'6');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(7,'77');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(8,'8','88');" + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column c text;" + random_restart + + # source1.tb1(a,c); source1.tb2(a,b); source2.tb1(a,c); source2.tb2(a,b,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(9,'999');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(10,'1010');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(11,'111111');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(12,'1212','121212');" + + run_sql_source2 "alter table ${shardddl1}.${tb2} drop column b;" + random_restart + + # source1.tb1(a,c); source1.tb2(a,b); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(13,'131313');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(14,'1414');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(15,'151515');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(16,'161616');" + + run_sql_source1 "alter table ${shardddl1}.${tb2} drop column b;" + random_restart + + # source1.tb1(a,c); source1.tb2(a); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(17,'171717');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(18);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(19,'191919');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(20,'202020');" + + run_sql_source1 "alter table ${shardddl1}.${tb2} add column c text;" + random_restart + + # source1.tb1(a,c); source1.tb2(a,c); source2.tb1(a,c); source2.tb2(a,c) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(21,'212121');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(22,'222222');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(23,'232323');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(24,'242424');" + + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 24" + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml +} + +function DM_RESTART_TASK_MASTER_WORKER() { + # create table with different schema, init data, and create table in downstream manually + run_case RESTART_TASK_MASTER_WORKER "double-source-optimistic" \ + "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key);\"; \ + run_sql_source1 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10));\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb1} (a int primary key, c text);\"; \ + run_sql_source2 \"create table ${shardddl1}.${tb2} (a int primary key, b varchar(10), c text);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb1} values(1);\"; \ + run_sql_source1 \"insert into ${shardddl1}.${tb2} values(2,'22');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb1} values(3,'333');\"; \ + run_sql_source2 \"insert into ${shardddl1}.${tb2} values(4,'44','444');\"; \ + run_sql_tidb \"create database if not exists ${shardddl};\"; \ + run_sql_tidb \"create table ${shardddl}.${tb} (a int primary key, b varchar(10), c text);\"" \ + "clean_table" "optimistic" +} + +function random_restart() { + mod=$(($RANDOM % 4)) + if [[ "$mod" == "0" ]]; then + echo "restart master" + restart_master + elif [[ "$mod" == "1" ]]; then + echo "restart worker1" + restart_worker1 + elif [[ "$mod" == "2" ]]; then + echo "restart worker2" + restart_worker2 + else + echo "restart task" + restart_task $cur/conf/double-source-optimistic.yaml + fi +} + +function DM_STOP_TASK_FOR_A_SOURCE_CASE() { + run_sql_source1 "insert into ${shardddl1}.${tb1} values(1);" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(2);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(3);" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(4);" + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column b varchar(10);" + run_sql_source1 "alter table ${shardddl1}.${tb2} add column b varchar(10);" + run_sql_source2 "alter table ${shardddl1}.${tb1} add column b varchar(10);" + run_sql_source2 "alter table ${shardddl1}.${tb2} add column b varchar(10);" + + run_sql_source1 "insert into ${shardddl1}.${tb1} values(5,'aaa');" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(6,'bbb');" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(7,'ccc');" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(8,'ddd');" + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column c text;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(9,'eee','eee');" + run_sql_source1 "alter table ${shardddl1}.${tb2} drop column b;" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(10);" + run_sql_source2 "alter table ${shardddl1}.${tb1} add column c text;" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(11,'fff','fff');" + run_sql_source2 "alter table ${shardddl1}.${tb2} drop column b;" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(12);" + + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 12" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-task test -s mysql-replica-02" \ + "\"result\": true" 2 + + run_sql_source1 "alter table ${shardddl1}.${tb1} drop column b;" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(13,'ggg');" + run_sql_source1 "alter table ${shardddl1}.${tb2} add column c text;" + run_sql_source1 "insert into ${shardddl1}.${tb2} values(14,'hhh');" + + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 14" + run_sql_tidb_with_retry "select count(1) from INFORMATION_SCHEMA.COLUMNS where TABLE_SCHEMA='${shardddl}' AND TABLE_NAME='${tb}';" \ + "count(1): 2" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $cur/conf/double-source-optimistic.yaml -s mysql-replica-02" \ + "\"result\": true" 2 + + run_sql_source1 "insert into ${shardddl1}.${tb1} values(15,'iii');" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(16,'jjj');" + run_sql_source2 "alter table ${shardddl1}.${tb1} drop column b;" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(17,'kkk');" + run_sql_source2 "alter table ${shardddl1}.${tb2} add column c text;" + run_sql_source2 "insert into ${shardddl1}.${tb2} values(18,'lll');" + + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml +} + +function DM_STOP_TASK_FOR_A_SOURCE() { + run_case STOP_TASK_FOR_A_SOURCE "double-source-optimistic" "init_table 111 112 211 212" "clean_table" "optimistic" +} + +function DM_UPDATE_BA_ROUTE_CASE() { + run_sql_source1 "insert into ${shardddl1}.${tb1} values(1);" + run_sql_source1 "insert into ${shardddl2}.${tb1} values(2);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(3);" + run_sql_source2 "insert into ${shardddl2}.${tb1} values(4);" + + run_sql_source1 "alter table ${shardddl1}.${tb1} add column new_col1 int" + run_sql_source1 "alter table ${shardddl2}.${tb1} add column new_col1 int" + run_sql_source2 "alter table ${shardddl1}.${tb1} add column new_col1 int" + run_sql_source2 "alter table ${shardddl2}.${tb1} add column new_col1 int" + + run_sql_source1 "insert into ${shardddl1}.${tb1} values(5,5);" + run_sql_source1 "insert into ${shardddl2}.${tb1} values(6,6);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(7,7);" + run_sql_source2 "insert into ${shardddl2}.${tb1} values(8,8);" + + # source1 db2.tb1 add column and then drop column + run_sql_source1 "alter table ${shardddl2}.${tb1} add column new_col2 int" + run_sql_source1 "insert into ${shardddl2}.${tb1} values(9,9,9);" + run_sql_source1 "alter table ${shardddl2}.${tb1} drop column new_col2" + run_sql_source1 "insert into ${shardddl2}.${tb1} values(10,10);" + + # source1.db1.tb1, source2.db2.tb1 add column + run_sql_source1 "alter table ${shardddl1}.${tb1} add column new_col3 int" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(11,11,11);" + run_sql_source2 "alter table ${shardddl2}.${tb1} add column new_col3 int" + run_sql_source2 "insert into ${shardddl2}.${tb1} values(12,12,12);" + + # source2 db1.tb1 drop column + run_sql_source2 "alter table ${shardddl1}.${tb1} drop column new_col1" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(13);" + + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb}" "count(1): 13" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-task test" \ + "\"result\": true" 3 + + sed 's/do-dbs: \["shardddl1","shardddl2"\]/do-dbs: \["shardddl1"\]/g' $cur/conf/double-source-optimistic.yaml >$WORK_DIR/task.yaml + + # source1: db1.tb1(id,new_col1,new_col3) + # source2: db1.tb1(id) + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $WORK_DIR/task.yaml" \ + "\"result\": true" 3 + + # no lock exist when task begin + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "show-ddl-locks" \ + "no DDL lock exists" 1 + + run_sql_source1 "insert into ${shardddl1}.${tb1} values(14,14,14);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(15);" + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb};" "count(1): 15" + + run_sql_source1 "alter table ${shardddl1}.${tb1} drop column new_col1" + run_sql_source2 "alter table ${shardddl1}.${tb1} add column new_col3 int" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(16,16);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(17,17);" + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb};" "count(1): 17" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-task test" \ + "\"result\": true" 3 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $cur/conf/double-source-optimistic.yaml" \ + "\"result\": true" 3 + + # source1: db1.tb1(id,new_col3), db2.tb1(id,new_col1) + # source2: db1.tb1(id,new_col3), db2.tb1(id,new_col1,new_col3) + run_sql_source1 "insert into ${shardddl1}.${tb1} values(18,18);" + run_sql_source1 "insert into ${shardddl2}.${tb1} values(19,19);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(20,20);" + run_sql_source2 "insert into ${shardddl2}.${tb1} values(21,21,21);" + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status test" \ + "Unknown column 'new_col1' in 'field list'" 2 + + run_sql_tidb "alter table ${shardddl}.${tb} add column new_col1 int" + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "resume-task test" \ + "\"result\": true" 3 + + run_sql_source1 "alter table ${shardddl2}.${tb1} drop column new_col1" + run_sql_source2 "alter table ${shardddl2}.${tb1} drop column new_col1" + run_sql_source1 "alter table ${shardddl2}.${tb1} add column new_col3 int" + + run_sql_source1 "insert into ${shardddl1}.${tb1} values(22,22);" + run_sql_source1 "insert into ${shardddl2}.${tb1} values(23,23);" + run_sql_source2 "insert into ${shardddl1}.${tb1} values(24,24);" + run_sql_source2 "insert into ${shardddl2}.${tb1} values(25,25);" + + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml +} + +function DM_UPDATE_BA_ROUTE() { + run_case UPDATE_BA_ROUTE "double-source-optimistic" "init_table 111 121 211 221" "clean_table" "optimistic" +} + +function run() { + init_cluster + init_database + + DM_DIFFERENT_SCHEMA_FULL + DM_DIFFERENT_SCHEMA_INCREMENTAL + DM_RESTART_TASK_MASTER_WORKER + DM_STOP_TASK_FOR_A_SOURCE + DM_UPDATE_BA_ROUTE +} + +cleanup_data $shardddl +cleanup_data $shardddl1 +cleanup_data $shardddl2 +# also cleanup dm processes in case of last run failed +cleanup_process $* +run $* +cleanup_process $* + +echo "[$(date)] <<<<<< test case $TEST_NAME success! >>>>>>" diff --git a/dm/tests/tls/conf/dm-task-2.yaml b/dm/tests/tls/conf/dm-task-2.yaml index 7c31d4ed9a2..27a0de091a3 100644 --- a/dm/tests/tls/conf/dm-task-2.yaml +++ b/dm/tests/tls/conf/dm-task-2.yaml @@ -47,6 +47,3 @@ syncers: global: worker-count: 16 batch: 100 - -tidb: - backend: "tidb" diff --git a/dm/tests/tls/conf/dm-task.yaml b/dm/tests/tls/conf/dm-task.yaml index eaae18aebd9..26e885afc9d 100644 --- a/dm/tests/tls/conf/dm-task.yaml +++ b/dm/tests/tls/conf/dm-task.yaml @@ -41,6 +41,3 @@ syncers: global: worker-count: 16 batch: 100 - -tidb: - backend: "tidb" diff --git a/dm/ui/server.go b/dm/ui/server.go index baeb5fd8c1f..722a6b7d2fb 100644 --- a/dm/ui/server.go +++ b/dm/ui/server.go @@ -18,6 +18,7 @@ package ui import ( "io/fs" "net/http" + "strings" "github.com/gin-gonic/gin" "github.com/pingcap/tiflow/dm/openapi" @@ -25,20 +26,41 @@ import ( "go.uber.org/zap" ) +const ( + buildPath = "dist" + assetsPath = "assets" + indexPath = "/dashboard/" +) + // WebUIAssetsHandler returns a http handler for serving static files. func WebUIAssetsHandler() http.FileSystem { - stripped, err := fs.Sub(WebUIAssets, "dist") + stripped, err := fs.Sub(WebUIAssets, buildPath) if err != nil { panic(err) // this should never happen } return http.FS(stripped) } +// we need this to handle this case: user want to access /dashboard/source.html/ but webui is a single page app, +// and it only can handle requests in index page, so we need to redirect to index page. +func alwaysRedirect(path string) gin.HandlerFunc { + return func(c *gin.Context) { + // note that static file like css and js under the assets folder should not be redirected. + if c.Request.URL.Path != path && !strings.Contains(c.Request.URL.Path, assetsPath) { + c.Redirect(http.StatusPermanentRedirect, path) + c.AbortWithStatus(http.StatusPermanentRedirect) + } else { + c.Next() + } + } +} + // InitWebUIRouter initializes the webUI router. func InitWebUIRouter() *gin.Engine { router := gin.New() router.Use(gin.Recovery()) + router.Use(alwaysRedirect(indexPath)) router.Use(openapi.ZapLogger(log.L().WithFields(zap.String("component", "webui")).Logger)) - router.StaticFS("/dashboard/", WebUIAssetsHandler()) + router.StaticFS(indexPath, WebUIAssetsHandler()) return router } diff --git a/dm/ui/yarn.lock b/dm/ui/yarn.lock index 3aaa56dfdef..b7e8c06c80b 100644 --- a/dm/ui/yarn.lock +++ b/dm/ui/yarn.lock @@ -2572,13 +2572,20 @@ is-ci@^3.0.1: dependencies: ci-info "^3.2.0" -is-core-module@^2.2.0, is-core-module@^2.8.0: +is-core-module@^2.2.0: version "2.8.0" resolved "https://registry.yarnpkg.com/is-core-module/-/is-core-module-2.8.0.tgz#0321336c3d0925e497fd97f5d95cb114a5ccd548" integrity sha512-vd15qHsaqrRL7dtH6QNuy0ndJmRDrS9HAM1CAiSifNUFv4x1a0CCVsj18hJ1mShxIG6T2i1sO78MkP56r0nYRw== dependencies: has "^1.0.3" +is-core-module@^2.8.0: + version "2.8.1" + resolved "https://registry.yarnpkg.com/is-core-module/-/is-core-module-2.8.1.tgz#f59fdfca701d5879d0a6b100a40aa1560ce27211" + integrity sha512-SdNCUs284hr40hFTFP6l0IfZ/RSrMXF3qgoRHd3/79unUTvrFO/JoXwkGm+5J/Oe3E/b5GsnG330uUNgRpu1PA== + dependencies: + has "^1.0.3" + is-date-object@^1.0.1: version "1.0.5" resolved "https://registry.yarnpkg.com/is-date-object/-/is-date-object-1.0.5.tgz#0841d5536e724c25597bf6ea62e1bd38298df31f" @@ -3126,9 +3133,9 @@ mute-stream@0.0.8: integrity sha512-nnbWWOkoWyUsTjKrhgD0dcz22mdkSnpYqbEjIm2nhwhuxlSkpywJmBo8h0ZqJdkp73mb90SssHkN4rsRaBAfAA== nanoid@^3.1.22, nanoid@^3.1.30: - version "3.1.30" - resolved "https://registry.yarnpkg.com/nanoid/-/nanoid-3.1.30.tgz#63f93cc548d2a113dc5dfbc63bfa09e2b9b64362" - integrity sha512-zJpuPDwOv8D2zq2WRoMe1HsfZthVewpel9CAvTfc/2mBD1uUT/agc5f7GHGWXlYkFvi1mVxe4IjvP2HNrop7nQ== + version "3.2.0" + resolved "https://registry.yarnpkg.com/nanoid/-/nanoid-3.2.0.tgz#62667522da6673971cca916a6d3eff3f415ff80c" + integrity sha512-fmsZYa9lpn69Ad5eDn7FMcnnSR+8R34W9qJEijxYhTbfOWzr22n1QxCMzXLK+ODyW2973V3Fux959iQoUxzUIA== natural-compare@^1.4.0: version "1.4.0" @@ -3160,9 +3167,9 @@ node-fetch-h2@^2.3.0: http2-client "^1.2.5" node-fetch@^2.6.1: - version "2.6.6" - resolved "https://registry.yarnpkg.com/node-fetch/-/node-fetch-2.6.6.tgz#1751a7c01834e8e1697758732e9efb6eeadfaf89" - integrity sha512-Z8/6vRlTUChSdIgMa51jxQ4lrw/Jy5SOW10ObaA47/RElsAN2c5Pn8bTgFGWn/ibwzXTE8qwr1Yzx28vsecXEA== + version "2.6.7" + resolved "https://registry.yarnpkg.com/node-fetch/-/node-fetch-2.6.7.tgz#24de9fba827e3b4ae44dc8b20256a379160052ad" + integrity sha512-ZjMPFEfVx5j+y2yF35Kzx5sF7kDzxuDj6ziH4FFbOp87zKDZNx8yExJIb05OGF4Nlt9IHFIMBkRl41VdvcNdbQ== dependencies: whatwg-url "^5.0.0" @@ -3475,7 +3482,7 @@ path-key@^3.0.0, path-key@^3.1.0: resolved "https://registry.yarnpkg.com/path-key/-/path-key-3.1.1.tgz#581f6ade658cbba65a0d3380de7753295054f375" integrity sha512-ojmeN0qd+y0jszEtoY48r0Peq5dwMEkIlCOu6Q5f41lfkswXuKtYrhgoTpLnyIcHm24Uhqx+5Tqm2InSwLhE6Q== -path-parse@^1.0.6: +path-parse@^1.0.6, path-parse@^1.0.7: version "1.0.7" resolved "https://registry.yarnpkg.com/path-parse/-/path-parse-1.0.7.tgz#fbc114b60ca42b30d9daf5858e4bd68bbedb6735" integrity sha512-LDJzPVEEEPR+y48z93A0Ed0yXb8pAByGWo/k5YYdYgpY2/2EsOsksJrq7lOHxryrVOn1ejG6oAp8ahvOIQD8sw== @@ -4212,7 +4219,16 @@ resolve-options@^1.1.0: dependencies: value-or-function "^3.0.0" -resolve@^1.1.6, resolve@^1.20.0: +resolve@^1.1.6: + version "1.21.0" + resolved "https://registry.yarnpkg.com/resolve/-/resolve-1.21.0.tgz#b51adc97f3472e6a5cf4444d34bc9d6b9037591f" + integrity sha512-3wCbTpk5WJlyE4mSOtDLhqQmGFi0/TD9VPwmiolnk8U0wRgMEktqCXd3vy5buTO3tljvalNvKrjHEfrd2WpEKA== + dependencies: + is-core-module "^2.8.0" + path-parse "^1.0.7" + supports-preserve-symlinks-flag "^1.0.0" + +resolve@^1.20.0: version "1.20.0" resolved "https://registry.yarnpkg.com/resolve/-/resolve-1.20.0.tgz#629a013fb3f70755d6f0b7935cc1c2c5378b1975" integrity sha512-wENBPt4ySzg4ybFQW2TT1zMQucPK95HSh/nq2CFTZVOGut2+pQvSsgtda4d26YrYcr067wjbmzOG8byDPBX63A== @@ -4378,9 +4394,9 @@ shebang-regex@^3.0.0: integrity sha512-7++dFhtcx3353uBaq8DDR4NuxBetBzC7ZQOhmTQInHEd6bSrXdiEyzCvG07Z44UYdLShWUyXt5M/yhz8ekcb1A== shelljs@^0.8.4: - version "0.8.4" - resolved "https://registry.yarnpkg.com/shelljs/-/shelljs-0.8.4.tgz#de7684feeb767f8716b326078a8a00875890e3c2" - integrity sha512-7gk3UZ9kOfPLIAbslLzyWeGiEqx9e3rxwZM0KE6EL8GlGwjym9Mrlx5/p33bWTu9YG6vcS4MBxYZDHYr5lr8BQ== + version "0.8.5" + resolved "https://registry.yarnpkg.com/shelljs/-/shelljs-0.8.5.tgz#de055408d8361bed66c669d2f000538ced8ee20c" + integrity sha512-TiwcRcrkhHvbrZbnRcFYMLl30Dfov3HKqzp5tO5b4pt6G/SezKcYhmDg15zXVBswHmctSAQKznqNW2LO5tTDow== dependencies: glob "^7.0.0" interpret "^1.0.0" @@ -4660,6 +4676,11 @@ supports-color@^9.0.2: resolved "https://registry.yarnpkg.com/supports-color/-/supports-color-9.2.1.tgz#599dc9d45acf74c6176e0d880bab1d7d718fe891" integrity sha512-Obv7ycoCTG51N7y175StI9BlAXrmgZrFhZOb0/PyjHBher/NmsdBgbbQ1Inhq+gIhz6+7Gb+jWF2Vqi7Mf1xnQ== +supports-preserve-symlinks-flag@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/supports-preserve-symlinks-flag/-/supports-preserve-symlinks-flag-1.0.0.tgz#6eda4bd344a3c94aea376d4cc31bc77311039e09" + integrity sha512-ot0WnXS9fgdkgIcePe6RHNk1WA8+muPa6cSjeR3V8K27q9BB1rTE3R1p7Hv0z1ZyAc8s6Vvv8DIyWf681MAt0w== + svg-path-properties@^0.2.1: version "0.2.2" resolved "https://registry.yarnpkg.com/svg-path-properties/-/svg-path-properties-0.2.2.tgz#b073d81be7292eae0e233ab8a83f58dc27113296" diff --git a/docs/design/2022-01-20-ticdc-mq-sink-multiple-topics.md b/docs/design/2022-01-20-ticdc-mq-sink-multiple-topics.md new file mode 100644 index 00000000000..318fecc0bd7 --- /dev/null +++ b/docs/design/2022-01-20-ticdc-mq-sink-multiple-topics.md @@ -0,0 +1,146 @@ +# TiCDC supports multi-topic dispatch + +- Author(s): [hi-rustin](https://github.com/hi-rustin) +- Tracking Issue: https://github.com/pingcap/tiflow/issues/4423 + +## Table of Contents + +- [Introduction](#introduction) +- [Motivation or Background](#motivation-or-background) +- [Detailed Design](#detailed-design) +- [Test Design](#test-design) + - [Functional Tests](#functional-tests) + - [Scenario Tests](#scenario-tests) + - [Compatibility Tests](#compatibility-tests) + - [Benchmark Tests](#benchmark-tests) +- [Impacts & Risks](#impacts--risks) +- [Investigation & Alternatives](#investigation--alternatives) +- [Unresolved Questions](#unresolved-questions) + +## Introduction + +This document provides a complete design on implementing multi-topic support in TiCDC MQ Sink. + +## Motivation or Background + +TiCDC MQ Sink only supports sending messages to a single topic, but in the MQ Sink usage scenario, we send data to +systems like [Flink], which requires us to support multiple topics, each topic as a data source. + +## Detailed Design + +This solution will introduce a new configuration to the configuration file that specifies which topic the sink will send +the table data to. + +We will continue to keep the original topic configuration in the sinkURI, which serves two purposes. + +1. when there is no new configuration or the configuration does not match, the data will be sent to that default topic. +2. DDLs of the schema level will be sent to this topic by default. + +### Topic dispatch configuration format + +This configuration will be added to the TiCDC changefeed configuration file. + +```toml +[sink] +dispatchers = [ + { matcher = ['test1.*', 'test2.*'], dispatcher = "ts", topic = "Topic dispatch expression" }, + { matcher = ['test3.*', 'test4.*'], dispatcher = "rowid", topic = "Topic dispatch expression" }, + { matcher = ['test1.*', 'test5.*'], dispatcher = "ts", topic = "Topic dispatch expression" }, +] +``` + +A new topic field has been added to dispatchers that will specify the topic dispatching rules for these tables. + +### Topic dispatch expression details + +The expression format looks like `flink_{schema}{table}`. This expression consists of two keywords and the `flink_` +prefix. + +Two keywords(case-insensitive): + +| Keyword | Description | Required | +| -------- | ---------------------- | -------- | +| {schema} | the name of the schema | no | +| {table} | the name of the table | no | + +> When neither keyword is filled in, it is equivalent to sending the data to a fixed topic. + +`flink_` is the user-definable part, where the user can fill in the expression with a custom string. + +Some examples: + +```toml +[sink] +dispatchers = [ + { matcher = ['test1.table1', 'test2.table1'], topic = "{schema}_{table}" }, + { matcher = ['test3.*', 'test4.*'], topic = "flink{schema}" }, + { matcher = ['test1.*', 'test5.*'], topic = "test-cdc" }, +] +``` + +- matcher = ['test1.*', 'test2.*'], topic = "{schema}\_{table}" + - Send the data from `test1.table1` and `test2.table1` to the `test1_table1` and `test2_table1` topics, respectively +- matcher = ['test3.*', 'test4.*'], topic = "flink\_{schema}" + - Send the data from all tables in `test3` and `test4` to `flinktest3` and `flinktest4` topics, respectively +- matcher = ['test1.*', 'test5.*'], topic = "test-cdc" + - Send the data of all the tables in `test1` (except `test1.table1`) and `test5` to the `test-cdc` topic + - The `table1` in `test1` is sent to the `test1_table1` topic, because for tables matching multiple matcher rules, the + topic expression corresponding to the top matcher prevails + +### DDL dispatch rules + +- schema-level DDLs that are sent to the default topic +- table-level DDLs, will be sent to the matching topic, if there is no topic match, it will be sent to the default topic + +## Test Design + +This functionality will be mainly covered by unit and integration tests. + +### Functional Tests + +#### Unit test + +Coverage should be more than 75% in new added code. + +#### Integration test + +Can pass all existing integration tests when changefeed without topic dispatch configuration. In addition, we will +integrate [Flink] into our integration tests to verify multi-topic functionality. + +### Scenario Tests + +We will test the scenario of using `canal-json` format to connect data to [Flink]. + +### Compatibility Tests + +#### Compatibility with other features/components + +For TiCDC's original support of only a single topic, we're not going to break it this time. When you pass only the +default topic in the sinkURI and there is no topic expression configuration, it will work as is. + +#### Upgrade compatibility + +When not configured, it works as a single topic, so just add the configuration and create a new changefeed after the +upgrade. + +#### Downgrade compatibility + +The new configuration is not recognized by the old TiCDC, so you need to remove the changefeed before downgrading. + +### Benchmark Tests + +N/A + +## Impacts & Risks + +N/A + +## Investigation & Alternatives + +N/A + +## Unresolved Questions + +N/A + +[flink]: https://flink.apache.org/ diff --git a/errors.toml b/errors.toml index 77e74567dae..8726dc63a35 100755 --- a/errors.toml +++ b/errors.toml @@ -143,7 +143,7 @@ changefeed update error: %s ["CDC:ErrCheckClusterVersionFromPD"] error = ''' -failed to request PD +failed to request PD %s, please try again later ''' ["CDC:ErrCheckDataDirSatisfied"] diff --git a/go.mod b/go.mod index 5d6bed4fd4e..bc9edb33616 100644 --- a/go.mod +++ b/go.mod @@ -35,6 +35,7 @@ require ( github.com/golang/protobuf v1.5.2 github.com/google/btree v1.0.0 github.com/google/go-cmp v0.5.6 + github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510 github.com/google/uuid v1.1.2 github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 @@ -53,12 +54,12 @@ require ( github.com/philhofer/fwd v1.0.0 // indirect github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712 github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c - github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd + github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 github.com/pingcap/kvproto v0.0.0-20211207042851-78a55fb8e69c github.com/pingcap/log v0.0.0-20211207084639-71a2e5860834 - github.com/pingcap/tidb v1.1.0-beta.0.20220112050342-d088e3d6fc6c + github.com/pingcap/tidb v1.1.0-beta.0.20220124083611-18fc286fbf0d github.com/pingcap/tidb-tools v5.2.3-0.20211105044302-2dabb6641a6e+incompatible - github.com/pingcap/tidb/parser v0.0.0-20220112091742-82a75542e83f + github.com/pingcap/tidb/parser v0.0.0-20220124083611-18fc286fbf0d github.com/prometheus/client_golang v1.7.1 github.com/prometheus/client_model v0.2.0 github.com/r3labs/diff v1.1.0 diff --git a/go.sum b/go.sum index ba80675aeb9..1f031ae2e48 100644 --- a/go.sum +++ b/go.sum @@ -89,7 +89,6 @@ github.com/Shopify/sarama v1.27.2 h1:1EyY1dsxNDUQEv0O/4TsjosHI2CgB1uo9H/v56xzTxc github.com/Shopify/sarama v1.27.2/go.mod h1:g5s5osgELxgM+Md9Qni9rzo7Rbt+vvFQI4bt/Mc93II= github.com/Shopify/toxiproxy v2.1.4+incompatible h1:TKdv8HiTLgE5wdJuEML90aBgNWsokNbMijUGhmcoBJc= github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI= -github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d h1:G0m3OIz70MZUWq3EgK3CesDbo8upS2Vm9/P3FtgI+Jk= github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= github.com/VividCortex/ewma v1.1.1 h1:MnEK4VOv6n0RSY4vtRe3h11qjxL3+t0B8yOL8iMXdcM= github.com/VividCortex/ewma v1.1.1/go.mod h1:2Tkkvm3sRDVXaiyucHiACn4cqf7DpdyLvmxzcbUokwA= @@ -334,8 +333,9 @@ github.com/go-logr/logr v0.1.0/go.mod h1:ixOQHD9gLJUVQQ2ZOR7zLEifBX6tGkNJF4QyIY7 github.com/go-martini/martini v0.0.0-20170121215854-22fa46961aab/go.mod h1:/P9AEU963A2AYjv4d1V5eVL1CQbEJq6aCNHDDjibzu8= github.com/go-mysql-org/go-mysql v1.4.1-0.20211217061939-06f932768788 h1:0IsP4ViNmA7ZElbCE4/lINdTppdw3jdcAiJaPDyeHx8= github.com/go-mysql-org/go-mysql v1.4.1-0.20211217061939-06f932768788/go.mod h1:3lFZKf7l95Qo70+3XB2WpiSf9wu2s3na3geLMaIIrqQ= -github.com/go-ole/go-ole v1.2.4 h1:nNBDSCOigTSiarFpYE9J/KtEA1IOW4CNeqT9TQDqCxI= github.com/go-ole/go-ole v1.2.4/go.mod h1:XCwSNxSkXRo4vlyPy93sltvi/qJq0jqQhjqQNIwKuxM= +github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY= +github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-openapi/jsonpointer v0.17.0/go.mod h1:cOnomiV+CVVwFLk0A/MExoFMjwdsUdVpsRhURCKh+3M= github.com/go-openapi/jsonpointer v0.19.2/go.mod h1:3akKfEdA7DF1sugOqz1dVQHBcuDBPKZGEoHC/NkiQRg= github.com/go-openapi/jsonpointer v0.19.3/go.mod h1:Pl9vOtqEWErmShwVjC8pYs9cog34VGT37dQOVbmoatg= @@ -486,6 +486,8 @@ github.com/google/pprof v0.0.0-20210720184732-4bb14d4b1be1 h1:K6RDEckDVWvDI9JAJY github.com/google/pprof v0.0.0-20210720184732-4bb14d4b1be1/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/shlex v0.0.0-20181106134648-c34317bd91bf/go.mod h1:RpwtwJQFrIEPstU94h88MWPXP2ektJZ8cZ0YntAmXiE= +github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510 h1:El6M4kTTCOh6aBiKaUGG7oYTSPP8MxqL4YI3kZKwcP4= +github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510/go.mod h1:pupxD2MaaD3pAXIBCelhxNneeOaAeabZDe5s4K6zSpQ= github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.2 h1:EVhdT+1Kseyi1/pUmXKaFxYsDNy9RQYkMWRH68J/W7Y= @@ -671,6 +673,8 @@ github.com/lib/pq v1.3.0 h1:/qkRGz8zljWiDcFvgpwUpwIAPu3r07TDvs3Rws+o/pU= github.com/lib/pq v1.3.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= github.com/linkedin/goavro/v2 v2.9.8 h1:jN50elxBsGBDGVDEKqUlDuU1cFwJ11K/yrJCBMe/7Wg= github.com/linkedin/goavro/v2 v2.9.8/go.mod h1:UgQUb2N/pmueQYH9bfqFioWxzYCZXSfF8Jw03O5sjqA= +github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 h1:6E+4a0GO5zZEnZ81pIr0yLvtUWk2if982qA3F3QD6H4= +github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/magiconair/properties v1.8.5/go.mod h1:y3VJvCyxH9uVvJTWEGAELF3aiYNyPKd5NZ3oSwXrF60= github.com/mailru/easyjson v0.0.0-20180823135443-60711f1a8329/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= @@ -815,8 +819,9 @@ github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c h1:xpW9bvK+HuuTm github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= github.com/pingcap/failpoint v0.0.0-20191029060244-12f4ac2fd11d/go.mod h1:DNS3Qg7bEDhU6EXNHF+XSv/PGznQaMJ5FWvctpm6pQI= github.com/pingcap/failpoint v0.0.0-20200702092429-9f69995143ce/go.mod h1:w4PEZ5y16LeofeeGwdgZB4ddv9bLyDuIX+ljstgKZyk= -github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd h1:I8IeI8MNiZVKnwuXhcIIzz6pREcOSbq18Q31KYIzFVM= github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd/go.mod h1:IVF+ijPSMZVtx2oIqxAg7ur6EyixtTYfOHwpfmlhqI4= +github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 h1:C3N3itkduZXDZFh4N3vQ5HEtld3S+Y+StULhWVvumU0= +github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 h1:Pe2LbxRmbTfAoKJ65bZLmhahmvHm7n9DUxGRQT00208= github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059/go.mod h1:fMRU1BA1y+r89AxUoaAar4JjrhUkVDt0o0Np6V8XbDQ= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= @@ -844,11 +849,11 @@ github.com/pingcap/parser v0.0.0-20210525032559-c37778aff307/go.mod h1:xZC8I7bug github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3/go.mod h1:tckvA041UWP+NqYzrJ3fMgC/Hw9wnmQ/tUkp/JaHly8= github.com/pingcap/sysutil v0.0.0-20210730114356-fcd8a63f68c5/go.mod h1:XsOaV712rUk63aOEKYP9PhXTIE3FMNHmC2r1wX5wElY= -github.com/pingcap/sysutil v0.0.0-20211208032423-041a72e5860d h1:k3/APKZjXOyJrFy8VyYwRlZhMelpD3qBLJNsw3bPl/g= -github.com/pingcap/sysutil v0.0.0-20211208032423-041a72e5860d/go.mod h1:7j18ezaWTao2LHOyMlsc2Dg1vW+mDY9dEbPzVyOlaeM= +github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 h1:HYbcxtnkN3s5tqrZ/z3eJS4j3Db8wMphEm1q10lY/TM= +github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4/go.mod h1:sDCsM39cGiv2vwunZkaFA917vVkqDTGSPbbV7z4Oops= github.com/pingcap/tidb v1.1.0-beta.0.20211023132847-efa94595c071/go.mod h1:Ci7ABF58a4jn6YtaHi7655jP409edqC2JxWWFRqOubg= -github.com/pingcap/tidb v1.1.0-beta.0.20220112050342-d088e3d6fc6c h1:NcpItOfJ3X0OSPUIY6DGS6QC79Ovcdb0TRXwbKHm15A= -github.com/pingcap/tidb v1.1.0-beta.0.20220112050342-d088e3d6fc6c/go.mod h1:z/k01yGpWuIx2qnM3/gPSOZ95JHAMBx51WXyWWUPH1w= +github.com/pingcap/tidb v1.1.0-beta.0.20220124083611-18fc286fbf0d h1:A7JcevlsUn4wd++wBF3re1rrEPzGdYYuq0zRtUTwRmE= +github.com/pingcap/tidb v1.1.0-beta.0.20220124083611-18fc286fbf0d/go.mod h1:VNhK7vWI4TEQhI8M9Pk06YAp1slz8iymaUKaVUrfqWE= github.com/pingcap/tidb-dashboard v0.0.0-20210312062513-eef5d6404638/go.mod h1:OzFN8H0EDMMqeulPhPMw2i2JaiZWOKFQ7zdRPhENNgo= github.com/pingcap/tidb-dashboard v0.0.0-20210716172320-2226872e3296/go.mod h1:OCXbZTBTIMRcIt0jFsuCakZP+goYRv6IjawKbwLS2TQ= github.com/pingcap/tidb-dashboard v0.0.0-20211008050453-a25c25809529/go.mod h1:OCXbZTBTIMRcIt0jFsuCakZP+goYRv6IjawKbwLS2TQ= @@ -859,8 +864,8 @@ github.com/pingcap/tidb-tools v5.2.3-0.20211105044302-2dabb6641a6e+incompatible github.com/pingcap/tidb-tools v5.2.3-0.20211105044302-2dabb6641a6e+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e/go.mod h1:e1MGCA9Sg3T8jid8PKAEq5eYVuMMCq4n8gJ+Kqp4Plg= github.com/pingcap/tidb/parser v0.0.0-20211023132847-efa94595c071/go.mod h1:e1MGCA9Sg3T8jid8PKAEq5eYVuMMCq4n8gJ+Kqp4Plg= -github.com/pingcap/tidb/parser v0.0.0-20220112091742-82a75542e83f h1:ZR9DHZXg02aFzmSVIqMEYwCG2VyTFk9gSk85u+RK4u8= -github.com/pingcap/tidb/parser v0.0.0-20220112091742-82a75542e83f/go.mod h1:ElJiub4lRy6UZDb+0JHDkGEdr6aOli+ykhyej7VCLoI= +github.com/pingcap/tidb/parser v0.0.0-20220124083611-18fc286fbf0d h1:QjRWklG/hLNibuK3jBk7uhnwGB3GM87f9MBw37dqOnw= +github.com/pingcap/tidb/parser v0.0.0-20220124083611-18fc286fbf0d/go.mod h1:ElJiub4lRy6UZDb+0JHDkGEdr6aOli+ykhyej7VCLoI= github.com/pingcap/tipb v0.0.0-20211008080435-3fd327dfce0e/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pingcap/tipb v0.0.0-20220107024056-3b91949a18a7 h1:DHU4vw0o15qdKsf7d/Pyhun4YtX8FwoDQxG0plPByUg= github.com/pingcap/tipb v0.0.0-20220107024056-3b91949a18a7/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= @@ -876,6 +881,8 @@ github.com/pkg/sftp v1.10.1/go.mod h1:lYOWFsE0bwd1+KfKJaKeuokY15vzFx25BLbzYYoAxZ github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= +github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c h1:ncq/mPwQF4JjgDlrVEn3C11VoGHZN7m8qihwgMEtzYw= +github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= github.com/prometheus/client_golang v0.9.0/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso= @@ -938,9 +945,10 @@ github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/shirou/gopsutil v2.19.10+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= +github.com/shirou/gopsutil v3.21.2+incompatible h1:U+YvJfjCh6MslYlIAXvPtzhW3YZEtc9uncueUNpD/0A= github.com/shirou/gopsutil v3.21.2+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= -github.com/shirou/gopsutil v3.21.3+incompatible h1:uenXGGa8ESCQq+dbgtl916dmg6PSAz2cXov0uORQ9v8= -github.com/shirou/gopsutil v3.21.3+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= +github.com/shirou/gopsutil/v3 v3.21.12 h1:VoGxEW2hpmz0Vt3wUvHIl9fquzYLNpVpgNNB7pGJimA= +github.com/shirou/gopsutil/v3 v3.21.12/go.mod h1:BToYZVTlSVlfazpDDYFnsVZLaoRG+g8ufT6fPQLdJzA= github.com/shirou/w32 v0.0.0-20160930032740-bb4de0191aa4/go.mod h1:qsXQc7+bwAM3Q1u/4XEfrquwF8Lw7D7y5cD8CuHnfIc= github.com/shopspring/decimal v0.0.0-20180709203117-cd690d0c9e24/go.mod h1:M+9NzErvs504Cn4c5DxATwIqPbtswREoFCre64PpcG4= github.com/shopspring/decimal v1.3.0 h1:KK3gWIXskZ2O1U/JNTisNcvH+jveJxZYrjbTsrbbnh8= @@ -1038,10 +1046,12 @@ github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee h1:rAAdvQ8Hh36syHr9 github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee/go.mod h1:lRbwxBAhnTQR5vqbTzeI/Bj62bD2OvYYuFezo2vrmeI= github.com/tinylib/msgp v1.1.0 h1:9fQd+ICuRIu/ue4vxJZu6/LzxN0HwMds2nq/0cFvxHU= github.com/tinylib/msgp v1.1.0/go.mod h1:+d+yLhGm8mzTaHzB+wgMYrodPfmZrzkirds8fDWklFE= -github.com/tklauser/go-sysconf v0.3.4 h1:HT8SVixZd3IzLdfs/xlpq0jeSfTX57g1v6wB1EuzV7M= github.com/tklauser/go-sysconf v0.3.4/go.mod h1:Cl2c8ZRWfHD5IrfHo9VN+FX9kCFjIOyVklgXycLB6ek= -github.com/tklauser/numcpus v0.2.1 h1:ct88eFm+Q7m2ZfXJdan1xYoXKlmwsfP+k88q05KvlZc= +github.com/tklauser/go-sysconf v0.3.9 h1:JeUVdAOWhhxVcU6Eqr/ATFHgXk/mmiItdKeJPev3vTo= +github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= github.com/tklauser/numcpus v0.2.1/go.mod h1:9aU+wOc6WjUIZEwWMP62PL/41d65P+iks1gBkr4QyP8= +github.com/tklauser/numcpus v0.3.0 h1:ILuRUQBtssgnxw0XXIjKUC56fgnOrFoQQ/4+DeU2biQ= +github.com/tklauser/numcpus v0.3.0/go.mod h1:yFGUr7TUHQRAhyqBcEg0Ge34zDBAsIvJJcyE6boqnA8= github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966 h1:j6JEOq5QWFker+d7mFQYOhjTZonQ7YkLTHm56dbn+yM= @@ -1116,6 +1126,8 @@ github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.1/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= +github.com/yusufpapurcu/wmi v1.2.2 h1:KBNDSne4vP5mbSWnJbO+51IMOXJB67QiYCSBrubbPRg= +github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.5 h1:XAzx9gjCb0Rxj7EoqcClPD1d5ZBxZJk0jbuoPHenBt0= @@ -1356,6 +1368,7 @@ golang.org/x/sys v0.0.0-20190801041406-cbf593c0f2f3/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191005200804-aed5e4c7ecf9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191010194322-b09406accb47/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1386,6 +1399,7 @@ golang.org/x/sys v0.0.0-20200905004654-be1d3432aa8f/go.mod h1:h1NjWce9XRLGQEsW7w 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= golang.org/x/sys v0.0.0-20201201145000-ef89a241ccb3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201204225414-ed752295db88/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210104204734-6f8348627aad/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210217105451-b926d437f341/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1403,8 +1417,10 @@ golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210616094352-59db8d763f22/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210806184541-e5e7981a1069/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210831042530-f4d43177bf5e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210927094055-39ccf1dd6fa6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211013075003-97ac67df715c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211019181941-9d821ace8654/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211031064116-611d5d643895/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e h1:fLOSk5Q00efkSvAm+4xcoXD+RRmLmmulPn5I3Y9F2EM= diff --git a/pkg/cmd/cli/cli_changefeed_create.go b/pkg/cmd/cli/cli_changefeed_create.go index 9f8667026f0..1f819a5ced8 100644 --- a/pkg/cmd/cli/cli_changefeed_create.go +++ b/pkg/cmd/cli/cli_changefeed_create.go @@ -163,17 +163,18 @@ func (o *createChangefeedOptions) complete(ctx context.Context, f factory.Factor } o.startTs = oracle.ComposeTS(ts, logical) } - - return o.completeCfg(ctx, cmd) -} - -// completeCfg complete the replica config from file and cmd flags. -func (o *createChangefeedOptions) completeCfg(ctx context.Context, cmd *cobra.Command) error { _, captureInfos, err := o.etcdClient.GetCaptures(ctx) if err != nil { return err } + return o.completeCfg(cmd, captureInfos) +} + +// completeCfg complete the replica config from file and cmd flags. +func (o *createChangefeedOptions) completeCfg( + cmd *cobra.Command, captureInfos []*model.CaptureInfo, +) error { cdcClusterVer, err := version.GetTiCDCClusterVersion(model.ListVersionsFromCaptureInfos(captureInfos)) if err != nil { return errors.Trace(err) @@ -227,6 +228,16 @@ func (o *createChangefeedOptions) completeCfg(ctx context.Context, cmd *cobra.Co } } + switch o.commonChangefeedOptions.sortEngine { + case model.SortInMemory: + case model.SortInFile: + case model.SortUnified: + default: + log.Warn("invalid sort-engine, use Unified Sorter by default", + zap.String("invalidSortEngine", o.commonChangefeedOptions.sortEngine)) + o.commonChangefeedOptions.sortEngine = model.SortUnified + } + if o.commonChangefeedOptions.sortEngine == model.SortUnified && !cdcClusterVer.ShouldEnableUnifiedSorterByDefault() { o.commonChangefeedOptions.sortEngine = model.SortInMemory log.Warn("The TiCDC cluster is built from an older version, disabling Unified Sorter by default", diff --git a/pkg/cmd/cli/cli_changefeed_create_test.go b/pkg/cmd/cli/cli_changefeed_create_test.go index 658346d38e6..6e877f617a9 100644 --- a/pkg/cmd/cli/cli_changefeed_create_test.go +++ b/pkg/cmd/cli/cli_changefeed_create_test.go @@ -20,8 +20,10 @@ import ( "testing" "github.com/pingcap/check" + "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/version" "github.com/spf13/cobra" ) @@ -65,3 +67,35 @@ func (s *changefeedSuite) TestStrictDecodeConfig(c *check.C) { c.Assert(err, check.NotNil) c.Assert(err, check.ErrorMatches, ".*CDC:ErrFilterRuleInvalid.*") } + +func (s *changefeedSuite) TestInvalidSortEngine(c *check.C) { + defer testleak.AfterTest(c)() + + cases := []struct { + input string + expect model.SortEngine + }{{ + input: "invalid", + expect: model.SortUnified, + }, { + input: "memory", + expect: model.SortInMemory, + }, { + input: "file", + expect: model.SortInFile, + }, { + input: "unified", + expect: model.SortUnified, + }} + for _, cs := range cases { + cmd := new(cobra.Command) + o := newChangefeedCommonOptions() + o.addFlags(cmd) + c.Assert(cmd.ParseFlags([]string{"--sort-engine=" + cs.input}), check.IsNil) + opt := newCreateChangefeedOptions(o) + err := opt.completeCfg(cmd, + []*model.CaptureInfo{{Version: version.MinTiCDCVersion.String()}}) + c.Assert(err, check.IsNil) + c.Assert(opt.commonChangefeedOptions.sortEngine, check.Equals, cs.expect) + } +} diff --git a/pkg/cmd/factory/factory_impl.go b/pkg/cmd/factory/factory_impl.go index fc4e9c77518..737e5e8f3ed 100644 --- a/pkg/cmd/factory/factory_impl.go +++ b/pkg/cmd/factory/factory_impl.go @@ -93,7 +93,8 @@ func (f *factoryImpl) EtcdClient() (*etcd.CDCEtcdClient, error) { } logConfig.Level = logLevel - pdEndpoints := strings.Split(f.GetPdAddr(), ",") + pdAddr := f.GetPdAddr() + pdEndpoints := strings.Split(pdAddr, ",") etcdClient, err := clientv3.New(clientv3.Config{ Context: ctx, @@ -118,7 +119,8 @@ func (f *factoryImpl) EtcdClient() (*etcd.CDCEtcdClient, error) { }, }) if err != nil { - return nil, err + return nil, errors.Annotatef(err, + "fail to open PD client, please check pd address \"%s\"", pdAddr) } client := etcd.NewCDCEtcdClient(ctx, etcdClient) @@ -156,7 +158,8 @@ func (f factoryImpl) PdClient() (pd.Client, error) { }), )) if err != nil { - return nil, errors.Annotatef(err, "fail to open PD client, pd=\"%s\"", pdAddr) + return nil, errors.Annotatef(err, + "fail to open PD client, please check pd address \"%s\"", pdAddr) } err = version.CheckClusterVersion(ctx, pdClient, pdEndpoints, credential, true) diff --git a/pkg/errors/errors.go b/pkg/errors/errors.go index 6b97fb1f893..5acab118024 100644 --- a/pkg/errors/errors.go +++ b/pkg/errors/errors.go @@ -117,7 +117,7 @@ var ( // utilities related errors ErrToTLSConfigFailed = errors.Normalize("generate tls config failed", errors.RFCCodeText("CDC:ErrToTLSConfigFailed")) - ErrCheckClusterVersionFromPD = errors.Normalize("failed to request PD", errors.RFCCodeText("CDC:ErrCheckClusterVersionFromPD")) + ErrCheckClusterVersionFromPD = errors.Normalize("failed to request PD %s, please try again later", errors.RFCCodeText("CDC:ErrCheckClusterVersionFromPD")) ErrNewSemVersion = errors.Normalize("create sem version", errors.RFCCodeText("CDC:ErrNewSemVersion")) ErrCheckDirWritable = errors.Normalize("check dir writable failed", errors.RFCCodeText("CDC:ErrCheckDirWritable")) ErrCheckDirReadable = errors.Normalize("check dir readable failed", errors.RFCCodeText("CDC:ErrCheckDirReadable")) diff --git a/pkg/orchestrator/etcd_worker_bank_test.go b/pkg/orchestrator/etcd_worker_bank_test.go index fbbcec988c5..f79f3c18edd 100644 --- a/pkg/orchestrator/etcd_worker_bank_test.go +++ b/pkg/orchestrator/etcd_worker_bank_test.go @@ -20,19 +20,19 @@ import ( "strconv" "strings" "sync" + "testing" "time" - "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/log" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/orchestrator/util" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" "go.uber.org/zap" ) type bankReactorState struct { - c *check.C + t *testing.T account []int pendingPatch [][]DataPatch index int @@ -42,7 +42,7 @@ type bankReactorState struct { const bankTestPrefix = "/ticdc/test/bank/" func (b *bankReactorState) Update(key util.EtcdKey, value []byte, isInit bool) error { - b.c.Assert(strings.HasPrefix(key.String(), bankTestPrefix), check.IsTrue) + require.True(b.t, strings.HasPrefix(key.String(), bankTestPrefix)) indexStr := key.String()[len(bankTestPrefix):] b.account[b.atoi(indexStr)] = b.atoi(string(value)) return nil @@ -62,12 +62,12 @@ func (b *bankReactorState) Check() { if sum != 0 { log.Info("show account", zap.Int("index", b.index), zap.Int("sum", sum), zap.Ints("account", b.account)) } - b.c.Assert(sum, check.Equals, 0, check.Commentf("not ft:%t", b.notFirstTick)) + require.Equal(b.t, sum, 0, fmt.Sprintf("not ft:%t", b.notFirstTick)) } func (b *bankReactorState) atoi(value string) int { i, err := strconv.Atoi(value) - b.c.Assert(err, check.IsNil) + require.Nil(b.t, err) return i } @@ -120,9 +120,7 @@ func (b *bankReactor) Tick(ctx context.Context, state ReactorState) (nextState R return state, err } -func (s *etcdWorkerSuite) TestEtcdBank(c *check.C) { - defer testleak.AfterTest(c)() - +func TestEtcdBank(t *testing.T) { _ = failpoint.Enable("github.com/pingcap/tiflow/pkg/orchestrator/InjectProgressRequestAfterCommit", "10%return(true)") defer func() { _ = failpoint.Disable("github.com/pingcap/tiflow/pkg/orchestrator/InjectProgressRequestAfterCommit") @@ -134,7 +132,7 @@ func (s *etcdWorkerSuite) TestEtcdBank(c *check.C) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - newClient, closer := setUpTest(c) + newClient, closer := setUpTest(t) defer closer() cli := newClient() @@ -144,7 +142,7 @@ func (s *etcdWorkerSuite) TestEtcdBank(c *check.C) { for i := 0; i < totalAccountNumber; i++ { _, err := cli.Put(ctx, fmt.Sprintf("%s%d", bankTestPrefix, i), "0") - c.Assert(err, check.IsNil) + require.Nil(t, err) } for i := 0; i < workerNumber; i++ { @@ -155,13 +153,13 @@ func (s *etcdWorkerSuite) TestEtcdBank(c *check.C) { for { worker, err := NewEtcdWorker(cli, bankTestPrefix, &bankReactor{ accountNumber: totalAccountNumber, - }, &bankReactorState{c: c, index: i, account: make([]int, totalAccountNumber)}) - c.Assert(err, check.IsNil) + }, &bankReactorState{t: t, index: i, account: make([]int, totalAccountNumber)}) + require.Nil(t, err) err = worker.Run(ctx, nil, 100*time.Millisecond, "127.0.0.1", "") if err == nil || err.Error() == "etcdserver: request timed out" { continue } - c.Assert(err, check.ErrorMatches, ".*context deadline exceeded.*") + require.Contains(t, err.Error(), "context deadline exceeded") return } }() diff --git a/pkg/orchestrator/etcd_worker_test.go b/pkg/orchestrator/etcd_worker_test.go index e65b463c54e..6d77450a012 100644 --- a/pkg/orchestrator/etcd_worker_test.go +++ b/pkg/orchestrator/etcd_worker_test.go @@ -16,6 +16,8 @@ package orchestrator import ( "context" "encoding/json" + "io/ioutil" + "os" "regexp" "strconv" "strings" @@ -23,14 +25,13 @@ import ( "testing" "time" - "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/log" cerrors "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/orchestrator/util" - "github.com/pingcap/tiflow/pkg/util/testleak" "github.com/prometheus/client_golang/prometheus" + "github.com/stretchr/testify/require" "go.etcd.io/etcd/clientv3" "go.uber.org/zap" "golang.org/x/sync/errgroup" @@ -43,12 +44,6 @@ const ( totalTicksPerReactor = 1000 ) -func Test(t *testing.T) { check.TestingT(t) } - -var _ = check.Suite(&etcdWorkerSuite{}) - -type etcdWorkerSuite struct{} - type simpleReactor struct { state *simpleReactorState tickCount int @@ -199,26 +194,27 @@ func (s *simpleReactorState) GetPatches() [][]DataPatch { return [][]DataPatch{ret} } -func setUpTest(c *check.C) (func() *etcd.Client, func()) { - dir := c.MkDir() +func setUpTest(t *testing.T) (func() *etcd.Client, func()) { + dir, err := ioutil.TempDir("", "etcd-test") + require.Nil(t, err) url, server, err := etcd.SetupEmbedEtcd(dir) - c.Assert(err, check.IsNil) + require.Nil(t, err) endpoints := []string{url.String()} return func() *etcd.Client { rawCli, err := clientv3.NewFromURLs(endpoints) - c.Check(err, check.IsNil) + require.Nil(t, err) return etcd.Wrap(rawCli, map[string]prometheus.Counter{}) }, func() { server.Close() + os.RemoveAll(dir) } } -func (s *etcdWorkerSuite) TestEtcdSum(c *check.C) { - defer testleak.AfterTest(c)() +func TestEtcdSum(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Minute*5) defer cancel() - newClient, closer := setUpTest(c) + newClient, closer := setUpTest(t) defer closer() cli := newClient() @@ -226,15 +222,15 @@ func (s *etcdWorkerSuite) TestEtcdSum(c *check.C) { _ = cli.Unwrap().Close() }() _, err := cli.Put(ctx, testEtcdKeyPrefix+"/sum", "0") - c.Check(err, check.IsNil) + require.Nil(t, err) initArray := make([]int, numValuesPerGroup) jsonStr, err := json.Marshal(initArray) - c.Check(err, check.IsNil) + require.Nil(t, err) for i := 0; i < numGroups; i++ { _, err := cli.Put(ctx, testEtcdKeyPrefix+"/"+strconv.Itoa(i), string(jsonStr)) - c.Check(err, check.IsNil) + require.Nil(t, err) } errg, ctx := errgroup.WithContext(ctx) @@ -278,7 +274,7 @@ func (s *etcdWorkerSuite) TestEtcdSum(c *check.C) { strings.Contains(err.Error(), "etcdserver: request timeout")) { return } - c.Check(err, check.IsNil) + require.Nil(t, err) } type intReactorState struct { @@ -327,20 +323,18 @@ func (r *linearizabilityReactor) Tick(ctx context.Context, state ReactorState) ( return r.state, nil } -func (s *etcdWorkerSuite) TestLinearizability(c *check.C) { - defer testleak.AfterTest(c)() - +func TestLinearizability(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Minute*5) defer cancel() - newClient, closer := setUpTest(c) + newClient, closer := setUpTest(t) defer closer() cli0 := newClient() cli := newClient() for i := 0; i < 1000; i++ { _, err := cli.Put(ctx, testEtcdKeyPrefix+"/lin", strconv.Itoa(i)) - c.Assert(err, check.IsNil) + require.Nil(t, err) } reactor, err := NewEtcdWorker(cli0, testEtcdKeyPrefix+"/lin", &linearizabilityReactor{ @@ -350,7 +344,7 @@ func (s *etcdWorkerSuite) TestLinearizability(c *check.C) { val: 0, isUpdated: false, }) - c.Assert(err, check.IsNil) + require.Nil(t, err) errg := &errgroup.Group{} errg.Go(func() error { return reactor.Run(ctx, nil, 10*time.Millisecond, "127.0.0.1", "") @@ -359,16 +353,16 @@ func (s *etcdWorkerSuite) TestLinearizability(c *check.C) { time.Sleep(500 * time.Millisecond) for i := 999; i < 2000; i++ { _, err := cli.Put(ctx, testEtcdKeyPrefix+"/lin", strconv.Itoa(i)) - c.Assert(err, check.IsNil) + require.Nil(t, err) } err = errg.Wait() - c.Assert(err, check.IsNil) + require.Nil(t, err) err = cli.Unwrap().Close() - c.Assert(err, check.IsNil) + require.Nil(t, err) err = cli0.Unwrap().Close() - c.Assert(err, check.IsNil) + require.Nil(t, err) } type commonReactorState struct { @@ -421,13 +415,11 @@ func (r *finishedReactor) Tick(ctx context.Context, state ReactorState) (nextSta return r.state, cerrors.ErrReactorFinished } -func (s *etcdWorkerSuite) TestFinished(c *check.C) { - defer testleak.AfterTest(c)() - +func TestFinished(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Minute*5) defer cancel() - newClient, closer := setUpTest(c) + newClient, closer := setUpTest(t) defer closer() cli := newClient() @@ -437,19 +429,18 @@ func (s *etcdWorkerSuite) TestFinished(c *check.C) { }, &commonReactorState{ state: make(map[string]string), }) - c.Assert(err, check.IsNil) - + require.Nil(t, err) err = reactor.Run(ctx, nil, 10*time.Millisecond, "127.0.0.1", "") - c.Assert(err, check.IsNil) + require.Nil(t, err) resp, err := cli.Get(ctx, prefix+"/key1") - c.Assert(err, check.IsNil) - c.Assert(string(resp.Kvs[0].Key), check.Equals, "/cdc_etcd_worker_test/finished/key1") - c.Assert(string(resp.Kvs[0].Value), check.Equals, "abcabcfin") + require.Nil(t, err) + require.Equal(t, string(resp.Kvs[0].Key), "/cdc_etcd_worker_test/finished/key1") + require.Equal(t, string(resp.Kvs[0].Value), "abcabcfin") resp, err = cli.Get(ctx, prefix+"/key2") - c.Assert(err, check.IsNil) - c.Assert(resp.Kvs, check.HasLen, 0) + require.Nil(t, err) + require.Len(t, resp.Kvs, 0) err = cli.Unwrap().Close() - c.Assert(err, check.IsNil) + require.Nil(t, err) } type coverReactor struct { @@ -491,13 +482,11 @@ func (r *coverReactor) Tick(ctx context.Context, state ReactorState) (nextState return r.state, cerrors.ErrReactorFinished } -func (s *etcdWorkerSuite) TestCover(c *check.C) { - defer testleak.AfterTest(c)() - +func TestCover(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Minute*5) defer cancel() - newClient, closer := setUpTest(c) + newClient, closer := setUpTest(t) defer closer() cli := newClient() @@ -507,20 +496,19 @@ func (s *etcdWorkerSuite) TestCover(c *check.C) { }, &commonReactorState{ state: make(map[string]string), }) - c.Assert(err, check.IsNil) - + require.Nil(t, err) err = reactor.Run(ctx, nil, 10*time.Millisecond, "127.0.0.1", "") - c.Assert(err, check.IsNil) + require.Nil(t, err) resp, err := cli.Get(ctx, prefix+"/key1") - c.Assert(err, check.IsNil) - c.Assert(string(resp.Kvs[0].Key), check.Equals, "/cdc_etcd_worker_test/cover/key1") - c.Assert(string(resp.Kvs[0].Value), check.Equals, "abccbaabccbafinfin") + require.Nil(t, err) + require.Equal(t, string(resp.Kvs[0].Key), "/cdc_etcd_worker_test/cover/key1") + require.Equal(t, string(resp.Kvs[0].Value), "abccbaabccbafinfin") resp, err = cli.Get(ctx, prefix+"/key2") - c.Assert(err, check.IsNil) - c.Assert(string(resp.Kvs[0].Key), check.Equals, "/cdc_etcd_worker_test/cover/key2") - c.Assert(string(resp.Kvs[0].Value), check.Equals, "fin") + require.Nil(t, err) + require.Equal(t, string(resp.Kvs[0].Key), "/cdc_etcd_worker_test/cover/key2") + require.Equal(t, string(resp.Kvs[0].Value), "fin") err = cli.Unwrap().Close() - c.Assert(err, check.IsNil) + require.Nil(t, err) } type emptyTxnReactor struct { @@ -570,13 +558,11 @@ func (r *emptyTxnReactor) Tick(ctx context.Context, state ReactorState) (nextSta return r.state, cerrors.ErrReactorFinished } -func (s *etcdWorkerSuite) TestEmptyTxn(c *check.C) { - defer testleak.AfterTest(c)() - +func TestEmptyTxn(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Minute*5) defer cancel() - newClient, closer := setUpTest(c) + newClient, closer := setUpTest(t) defer closer() cli := newClient() @@ -587,19 +573,18 @@ func (s *etcdWorkerSuite) TestEmptyTxn(c *check.C) { }, &commonReactorState{ state: make(map[string]string), }) - c.Assert(err, check.IsNil) - + require.Nil(t, err) err = reactor.Run(ctx, nil, 10*time.Millisecond, "127.0.0.1", "") - c.Assert(err, check.IsNil) + require.Nil(t, err) resp, err := cli.Get(ctx, prefix+"/key1") - c.Assert(err, check.IsNil) - c.Assert(resp.Kvs, check.HasLen, 0) + require.Nil(t, err) + require.Len(t, resp.Kvs, 0) resp, err = cli.Get(ctx, prefix+"/key2") - c.Assert(err, check.IsNil) - c.Assert(string(resp.Kvs[0].Key), check.Equals, "/cdc_etcd_worker_test/empty_txn/key2") - c.Assert(string(resp.Kvs[0].Value), check.Equals, "123") + require.Nil(t, err) + require.Equal(t, string(resp.Kvs[0].Key), "/cdc_etcd_worker_test/empty_txn/key2") + require.Equal(t, string(resp.Kvs[0].Value), "123") err = cli.Unwrap().Close() - c.Assert(err, check.IsNil) + require.Nil(t, err) } type emptyOrNilReactor struct { @@ -639,13 +624,11 @@ func (r *emptyOrNilReactor) Tick(ctx context.Context, state ReactorState) (nextS return r.state, cerrors.ErrReactorFinished } -func (s *etcdWorkerSuite) TestEmptyOrNil(c *check.C) { - defer testleak.AfterTest(c)() - +func TestEmptyOrNil(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Minute*5) defer cancel() - newClient, closer := setUpTest(c) + newClient, closer := setUpTest(t) defer closer() cli := newClient() @@ -655,19 +638,18 @@ func (s *etcdWorkerSuite) TestEmptyOrNil(c *check.C) { }, &commonReactorState{ state: make(map[string]string), }) - c.Assert(err, check.IsNil) - + require.Nil(t, err) err = reactor.Run(ctx, nil, 10*time.Millisecond, "127.0.0.1", "") - c.Assert(err, check.IsNil) + require.Nil(t, err) resp, err := cli.Get(ctx, prefix+"/key1") - c.Assert(err, check.IsNil) - c.Assert(string(resp.Kvs[0].Key), check.Equals, "/cdc_etcd_worker_test/emptyOrNil/key1") - c.Assert(string(resp.Kvs[0].Value), check.Equals, "") + require.Nil(t, err) + require.Equal(t, string(resp.Kvs[0].Key), "/cdc_etcd_worker_test/emptyOrNil/key1") + require.Equal(t, string(resp.Kvs[0].Value), "") resp, err = cli.Get(ctx, prefix+"/key2") - c.Assert(err, check.IsNil) - c.Assert(resp.Kvs, check.HasLen, 0) + require.Nil(t, err) + require.Len(t, resp.Kvs, 0) err = cli.Unwrap().Close() - c.Assert(err, check.IsNil) + require.Nil(t, err) } type modifyOneReactor struct { @@ -709,20 +691,18 @@ func (r *modifyOneReactor) Tick(ctx context.Context, state ReactorState) (nextSt // TestModifyAfterDelete tests snapshot isolation when there is one modifying transaction delayed in the middle while a deleting transaction // commits. The first transaction should be aborted and retried, and isolation should not be violated. -func (s *etcdWorkerSuite) TestModifyAfterDelete(c *check.C) { - defer testleak.AfterTest(c)() - +func TestModifyAfterDelete(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Minute*5) defer cancel() - newClient, closer := setUpTest(c) + newClient, closer := setUpTest(t) defer closer() cli1 := newClient() cli2 := newClient() _, err := cli1.Put(ctx, "/test/key1", "original value") - c.Assert(err, check.IsNil) + require.Nil(t, err) modifyReactor := &modifyOneReactor{ key: []byte("/test/key1"), @@ -732,14 +712,14 @@ func (s *etcdWorkerSuite) TestModifyAfterDelete(c *check.C) { worker1, err := NewEtcdWorker(cli1, "/test", modifyReactor, &commonReactorState{ state: make(map[string]string), }) - c.Assert(err, check.IsNil) + require.Nil(t, err) var wg sync.WaitGroup wg.Add(1) go func() { defer wg.Done() err := worker1.Run(ctx, nil, time.Millisecond*100, "127.0.0.1", "") - c.Assert(err, check.IsNil) + require.Nil(t, err) }() modifyReactor.waitOnCh <- struct{}{} @@ -751,18 +731,18 @@ func (s *etcdWorkerSuite) TestModifyAfterDelete(c *check.C) { worker2, err := NewEtcdWorker(cli2, "/test", deleteReactor, &commonReactorState{ state: make(map[string]string), }) - c.Assert(err, check.IsNil) + require.Nil(t, err) err = worker2.Run(ctx, nil, time.Millisecond*100, "127.0.0.1", "") - c.Assert(err, check.IsNil) + require.Nil(t, err) modifyReactor.waitOnCh <- struct{}{} wg.Wait() resp, err := cli1.Get(ctx, "/test/key1") - c.Assert(err, check.IsNil) - c.Assert(resp.Kvs, check.HasLen, 0) - c.Assert(worker1.deleteCounter, check.Equals, int64(1)) + require.Nil(t, err) + require.Len(t, resp.Kvs, 0) + require.Equal(t, worker1.deleteCounter, int64(1)) _ = cli1.Unwrap().Close() _ = cli2.Unwrap().Close() diff --git a/pkg/orchestrator/main_test.go b/pkg/orchestrator/main_test.go new file mode 100644 index 00000000000..5b44e017491 --- /dev/null +++ b/pkg/orchestrator/main_test.go @@ -0,0 +1,24 @@ +// Copyright 2022 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 orchestrator + +import ( + "testing" + + "github.com/pingcap/tiflow/pkg/leakutil" +) + +func TestMain(m *testing.M) { + leakutil.SetUpLeakTest(m) +} diff --git a/pkg/orchestrator/reactor_state_test.go b/pkg/orchestrator/reactor_state_test.go index 8d51eaa6f4d..442cfec0386 100644 --- a/pkg/orchestrator/reactor_state_test.go +++ b/pkg/orchestrator/reactor_state_test.go @@ -15,38 +15,33 @@ package orchestrator import ( "encoding/json" + "fmt" + "testing" "time" "github.com/google/go-cmp/cmp" "github.com/google/go-cmp/cmp/cmpopts" - "github.com/pingcap/check" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/orchestrator/util" - "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) -type stateSuite struct{} - -var _ = check.Suite(&stateSuite{}) - -func (s *stateSuite) TestCheckCaptureAlive(c *check.C) { - defer testleak.AfterTest(c)() +func TestCheckCaptureAlive(t *testing.T) { state := NewChangefeedReactorState("test") - stateTester := NewReactorStateTester(c, state, nil) + stateTester := NewReactorStateTester(t, state, nil) state.CheckCaptureAlive("6bbc01c8-0605-4f86-a0f9-b3119109b225") - c.Assert(stateTester.ApplyPatches(), check.ErrorMatches, ".*[CDC:ErrLeaseExpired].*") + require.Contains(t, stateTester.ApplyPatches().Error(), "[CDC:ErrLeaseExpired]") err := stateTester.Update("/tidb/cdc/capture/6bbc01c8-0605-4f86-a0f9-b3119109b225", []byte(`{"id":"6bbc01c8-0605-4f86-a0f9-b3119109b225","address":"127.0.0.1:8300"}`)) - c.Assert(err, check.IsNil) + require.Nil(t, err) state.CheckCaptureAlive("6bbc01c8-0605-4f86-a0f9-b3119109b225") stateTester.MustApplyPatches() } -func (s *stateSuite) TestChangefeedStateUpdate(c *check.C) { - defer testleak.AfterTest(c)() +func TestChangefeedStateUpdate(t *testing.T) { createTime, err := time.Parse("2006-01-02", "2020-02-02") - c.Assert(err, check.IsNil) + require.Nil(t, err) testCases := []struct { changefeedID string updateKey []string @@ -320,24 +315,23 @@ func (s *stateSuite) TestChangefeedStateUpdate(c *check.C) { value = nil } err = state.Update(util.NewEtcdKey(k), value, false) - c.Assert(err, check.IsNil) + require.Nil(t, err) } - c.Assert(cmp.Equal(state, &tc.expected, cmpopts.IgnoreUnexported(ChangefeedReactorState{})), check.IsTrue, - check.Commentf("%d,%s", i, cmp.Diff(state, &tc.expected, cmpopts.IgnoreUnexported(ChangefeedReactorState{})))) + require.True(t, cmp.Equal(state, &tc.expected, cmpopts.IgnoreUnexported(ChangefeedReactorState{})), + fmt.Sprintf("%d,%s", i, cmp.Diff(state, &tc.expected, cmpopts.IgnoreUnexported(ChangefeedReactorState{})))) } } -func (s *stateSuite) TestPatchInfo(c *check.C) { - defer testleak.AfterTest(c)() +func TestPatchInfo(t *testing.T) { state := NewChangefeedReactorState("test1") - stateTester := NewReactorStateTester(c, state, nil) + stateTester := NewReactorStateTester(t, state, nil) state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { - c.Assert(info, check.IsNil) + require.Nil(t, info) return &model.ChangeFeedInfo{SinkURI: "123", Config: &config.ReplicaConfig{}}, true, nil }) stateTester.MustApplyPatches() defaultConfig := config.GetDefaultReplicaConfig() - c.Assert(state.Info, check.DeepEquals, &model.ChangeFeedInfo{ + require.Equal(t, state.Info, &model.ChangeFeedInfo{ SinkURI: "123", Engine: model.SortUnified, Config: &config.ReplicaConfig{ @@ -354,7 +348,7 @@ func (s *stateSuite) TestPatchInfo(c *check.C) { return info, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.Info, check.DeepEquals, &model.ChangeFeedInfo{ + require.Equal(t, state.Info, &model.ChangeFeedInfo{ SinkURI: "123", StartTs: 6, Engine: model.SortUnified, @@ -371,52 +365,50 @@ func (s *stateSuite) TestPatchInfo(c *check.C) { return nil, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.Info, check.IsNil) + require.Nil(t, state.Info) } -func (s *stateSuite) TestPatchStatus(c *check.C) { - defer testleak.AfterTest(c)() +func TestPatchStatus(t *testing.T) { state := NewChangefeedReactorState("test1") - stateTester := NewReactorStateTester(c, state, nil) + stateTester := NewReactorStateTester(t, state, nil) state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { - c.Assert(status, check.IsNil) + require.Nil(t, status) return &model.ChangeFeedStatus{CheckpointTs: 5}, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.Status, check.DeepEquals, &model.ChangeFeedStatus{CheckpointTs: 5}) + require.Equal(t, state.Status, &model.ChangeFeedStatus{CheckpointTs: 5}) state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { status.ResolvedTs = 6 return status, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.Status, check.DeepEquals, &model.ChangeFeedStatus{CheckpointTs: 5, ResolvedTs: 6}) + require.Equal(t, state.Status, &model.ChangeFeedStatus{CheckpointTs: 5, ResolvedTs: 6}) state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { return nil, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.Status, check.IsNil) + require.Nil(t, state.Status) } -func (s *stateSuite) TestPatchTaskPosition(c *check.C) { - defer testleak.AfterTest(c)() +func TestPatchTaskPosition(t *testing.T) { state := NewChangefeedReactorState("test1") - stateTester := NewReactorStateTester(c, state, nil) + stateTester := NewReactorStateTester(t, state, nil) captureID1 := "capture1" captureID2 := "capture2" state.PatchTaskPosition(captureID1, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { - c.Assert(position, check.IsNil) + require.Nil(t, position) return &model.TaskPosition{ CheckPointTs: 1, }, true, nil }) state.PatchTaskPosition(captureID2, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { - c.Assert(position, check.IsNil) + require.Nil(t, position) return &model.TaskPosition{ CheckPointTs: 2, }, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.TaskPositions, check.DeepEquals, map[string]*model.TaskPosition{ + require.Equal(t, state.TaskPositions, map[string]*model.TaskPosition{ captureID1: { CheckPointTs: 1, }, @@ -433,7 +425,7 @@ func (s *stateSuite) TestPatchTaskPosition(c *check.C) { return position, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.TaskPositions, check.DeepEquals, map[string]*model.TaskPosition{ + require.Equal(t, state.TaskPositions, map[string]*model.TaskPosition{ captureID1: { CheckPointTs: 3, }, @@ -453,7 +445,7 @@ func (s *stateSuite) TestPatchTaskPosition(c *check.C) { return position, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.TaskPositions, check.DeepEquals, map[string]*model.TaskPosition{ + require.Equal(t, state.TaskPositions, map[string]*model.TaskPosition{ captureID1: { CheckPointTs: 3, Count: 6, @@ -461,26 +453,25 @@ func (s *stateSuite) TestPatchTaskPosition(c *check.C) { }) } -func (s *stateSuite) TestPatchTaskStatus(c *check.C) { - defer testleak.AfterTest(c)() +func TestPatchTaskStatus(t *testing.T) { state := NewChangefeedReactorState("test1") - stateTester := NewReactorStateTester(c, state, nil) + stateTester := NewReactorStateTester(t, state, nil) captureID1 := "capture1" captureID2 := "capture2" state.PatchTaskStatus(captureID1, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { - c.Assert(status, check.IsNil) + require.Nil(t, status) return &model.TaskStatus{ Tables: map[model.TableID]*model.TableReplicaInfo{45: {StartTs: 1}}, }, true, nil }) state.PatchTaskStatus(captureID2, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { - c.Assert(status, check.IsNil) + require.Nil(t, status) return &model.TaskStatus{ Tables: map[model.TableID]*model.TableReplicaInfo{46: {StartTs: 1}}, }, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.TaskStatuses, check.DeepEquals, map[model.CaptureID]*model.TaskStatus{ + require.Equal(t, state.TaskStatuses, map[model.CaptureID]*model.TaskStatus{ captureID1: {Tables: map[model.TableID]*model.TableReplicaInfo{45: {StartTs: 1}}}, captureID2: {Tables: map[model.TableID]*model.TableReplicaInfo{46: {StartTs: 1}}}, }) @@ -493,7 +484,7 @@ func (s *stateSuite) TestPatchTaskStatus(c *check.C) { return status, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.TaskStatuses, check.DeepEquals, map[model.CaptureID]*model.TaskStatus{ + require.Equal(t, state.TaskStatuses, map[model.CaptureID]*model.TaskStatus{ captureID1: {Tables: map[model.TableID]*model.TableReplicaInfo{45: {StartTs: 1}, 46: {StartTs: 2}}}, captureID2: {Tables: map[model.TableID]*model.TableReplicaInfo{46: {StartTs: 2}}}, }) @@ -501,27 +492,26 @@ func (s *stateSuite) TestPatchTaskStatus(c *check.C) { return nil, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.TaskStatuses, check.DeepEquals, map[model.CaptureID]*model.TaskStatus{ + require.Equal(t, state.TaskStatuses, map[model.CaptureID]*model.TaskStatus{ captureID1: {Tables: map[model.TableID]*model.TableReplicaInfo{45: {StartTs: 1}, 46: {StartTs: 2}}}, }) } -func (s *stateSuite) TestPatchTaskWorkload(c *check.C) { - defer testleak.AfterTest(c)() +func TestPatchTaskWorkload(t *testing.T) { state := NewChangefeedReactorState("test1") - stateTester := NewReactorStateTester(c, state, nil) + stateTester := NewReactorStateTester(t, state, nil) captureID1 := "capture1" captureID2 := "capture2" state.PatchTaskWorkload(captureID1, func(workload model.TaskWorkload) (model.TaskWorkload, bool, error) { - c.Assert(workload, check.IsNil) + require.Nil(t, workload) return model.TaskWorkload{45: {Workload: 1}}, true, nil }) state.PatchTaskWorkload(captureID2, func(workload model.TaskWorkload) (model.TaskWorkload, bool, error) { - c.Assert(workload, check.IsNil) + require.Nil(t, workload) return model.TaskWorkload{46: {Workload: 1}}, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.Workloads, check.DeepEquals, map[model.CaptureID]model.TaskWorkload{ + require.Equal(t, state.Workloads, map[model.CaptureID]model.TaskWorkload{ captureID1: {45: {Workload: 1}}, captureID2: {46: {Workload: 1}}, }) @@ -534,7 +524,7 @@ func (s *stateSuite) TestPatchTaskWorkload(c *check.C) { return workload, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.Workloads, check.DeepEquals, map[model.CaptureID]model.TaskWorkload{ + require.Equal(t, state.Workloads, map[model.CaptureID]model.TaskWorkload{ captureID1: {45: {Workload: 1}, 46: {Workload: 2}}, captureID2: {45: {Workload: 3}, 46: {Workload: 1}}, }) @@ -542,13 +532,12 @@ func (s *stateSuite) TestPatchTaskWorkload(c *check.C) { return nil, true, nil }) stateTester.MustApplyPatches() - c.Assert(state.Workloads, check.DeepEquals, map[model.CaptureID]model.TaskWorkload{ + require.Equal(t, state.Workloads, map[model.CaptureID]model.TaskWorkload{ captureID1: {45: {Workload: 1}, 46: {Workload: 2}}, }) } -func (s *stateSuite) TestGlobalStateUpdate(c *check.C) { - defer testleak.AfterTest(c)() +func TestGlobalStateUpdate(t *testing.T) { testCases := []struct { updateKey []string updateValue []string @@ -647,26 +636,25 @@ func (s *stateSuite) TestGlobalStateUpdate(c *check.C) { value = nil } err := state.Update(util.NewEtcdKey(k), value, false) - c.Assert(err, check.IsNil) + require.Nil(t, err) } - c.Assert(cmp.Equal(state, &tc.expected, cmpopts.IgnoreUnexported(GlobalReactorState{}, ChangefeedReactorState{})), check.IsTrue, - check.Commentf("%s", cmp.Diff(state, &tc.expected, cmpopts.IgnoreUnexported(GlobalReactorState{}, ChangefeedReactorState{})))) + require.True(t, cmp.Equal(state, &tc.expected, cmpopts.IgnoreUnexported(GlobalReactorState{}, ChangefeedReactorState{})), + cmp.Diff(state, &tc.expected, cmpopts.IgnoreUnexported(GlobalReactorState{}, ChangefeedReactorState{}))) } } -func (s *stateSuite) TestCaptureChangeHooks(c *check.C) { - defer testleak.AfterTest(c)() +func TestCaptureChangeHooks(t *testing.T) { state := NewGlobalState() var callCount int state.onCaptureAdded = func(captureID model.CaptureID, addr string) { callCount++ - c.Check(captureID, check.Equals, "capture-1") - c.Check(addr, check.Equals, "ip-1:8300") + require.Equal(t, captureID, "capture-1") + require.Equal(t, addr, "ip-1:8300") } state.onCaptureRemoved = func(captureID model.CaptureID) { callCount++ - c.Check(captureID, check.Equals, "capture-1") + require.Equal(t, captureID, "capture-1") } captureInfo := &model.CaptureInfo{ @@ -674,21 +662,20 @@ func (s *stateSuite) TestCaptureChangeHooks(c *check.C) { AdvertiseAddr: "ip-1:8300", } captureInfoBytes, err := json.Marshal(captureInfo) - c.Check(err, check.IsNil) + require.Nil(t, err) err = state.Update(util.NewEtcdKey(etcd.CaptureInfoKeyPrefix+"/capture-1"), captureInfoBytes, false) - c.Check(err, check.IsNil) - c.Check(callCount, check.Equals, 1) + require.Nil(t, err) + require.Equal(t, callCount, 1) err = state.Update(util.NewEtcdKey(etcd.CaptureInfoKeyPrefix+"/capture-1"), nil /* delete */, false) - c.Check(err, check.IsNil) - c.Check(callCount, check.Equals, 2) + require.Nil(t, err) + require.Equal(t, callCount, 2) } -func (s *stateSuite) TestCheckChangefeedNormal(c *check.C) { - defer testleak.AfterTest(c)() +func TestCheckChangefeedNormal(t *testing.T) { state := NewChangefeedReactorState("test1") - stateTester := NewReactorStateTester(c, state, nil) + stateTester := NewReactorStateTester(t, state, nil) state.CheckChangefeedNormal() stateTester.MustApplyPatches() state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { @@ -699,7 +686,7 @@ func (s *stateSuite) TestCheckChangefeedNormal(c *check.C) { }) state.CheckChangefeedNormal() stateTester.MustApplyPatches() - c.Assert(state.Status.ResolvedTs, check.Equals, uint64(1)) + require.Equal(t, state.Status.ResolvedTs, uint64(1)) state.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { info.AdminJobType = model.AdminStop @@ -711,7 +698,7 @@ func (s *stateSuite) TestCheckChangefeedNormal(c *check.C) { }) state.CheckChangefeedNormal() stateTester.MustApplyPatches() - c.Assert(state.Status.ResolvedTs, check.Equals, uint64(1)) + require.Equal(t, state.Status.ResolvedTs, uint64(1)) state.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) { status.ResolvedTs = 2 @@ -719,5 +706,5 @@ func (s *stateSuite) TestCheckChangefeedNormal(c *check.C) { }) state.CheckChangefeedNormal() stateTester.MustApplyPatches() - c.Assert(state.Status.ResolvedTs, check.Equals, uint64(2)) + require.Equal(t, state.Status.ResolvedTs, uint64(2)) } diff --git a/pkg/orchestrator/reactor_state_tester.go b/pkg/orchestrator/reactor_state_tester.go index 72b5f5e2c0a..5e724616697 100644 --- a/pkg/orchestrator/reactor_state_tester.go +++ b/pkg/orchestrator/reactor_state_tester.go @@ -14,55 +14,57 @@ package orchestrator import ( - "github.com/pingcap/check" + "testing" + "github.com/pingcap/errors" cerrors "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/orchestrator/util" + "github.com/stretchr/testify/require" ) // ReactorStateTester is a helper struct for unit-testing an implementer of ReactorState type ReactorStateTester struct { - c *check.C + t *testing.T state ReactorState kvEntries map[string]string } // NewReactorStateTester creates a new ReactorStateTester -func NewReactorStateTester(c *check.C, state ReactorState, initKVEntries map[string]string) *ReactorStateTester { +func NewReactorStateTester(t *testing.T, state ReactorState, initKVEntries map[string]string) *ReactorStateTester { if initKVEntries == nil { initKVEntries = make(map[string]string) } for k, v := range initKVEntries { err := state.Update(util.NewEtcdKey(k), []byte(v), true) - c.Assert(err, check.IsNil) + require.NoError(t, err) } return &ReactorStateTester{ - c: c, + t: t, state: state, kvEntries: initKVEntries, } } // Update is used to update keys in the mocked kv-store. -func (t *ReactorStateTester) Update(key string, value []byte) error { +func (r *ReactorStateTester) Update(key string, value []byte) error { k := util.NewEtcdKey(key) - err := t.state.Update(k, value, false) + err := r.state.Update(k, value, false) if err != nil { return errors.Trace(err) } if value != nil { - t.kvEntries[key] = string(value) + r.kvEntries[key] = string(value) } else { - delete(t.kvEntries, key) + delete(r.kvEntries, key) } return nil } // ApplyPatches calls the GetPatches method on the ReactorState and apply the changes to the mocked kv-store. -func (t *ReactorStateTester) ApplyPatches() error { - patchGroups := t.state.GetPatches() +func (r *ReactorStateTester) ApplyPatches() error { + patchGroups := r.state.GetPatches() for _, patches := range patchGroups { - err := t.applyPatches(patches) + err := r.applyPatches(patches) if err != nil { return err } @@ -70,11 +72,11 @@ func (t *ReactorStateTester) ApplyPatches() error { return nil } -func (t *ReactorStateTester) applyPatches(patches []DataPatch) error { +func (r *ReactorStateTester) applyPatches(patches []DataPatch) error { RetryLoop: for { tmpKVEntries := make(map[util.EtcdKey][]byte) - for k, v := range t.kvEntries { + for k, v := range r.kvEntries { tmpKVEntries[util.NewEtcdKey(k)] = []byte(v) } changedSet := make(map[util.EtcdKey]struct{}) @@ -89,14 +91,14 @@ RetryLoop: } } for k := range changedSet { - err := t.state.Update(k, tmpKVEntries[k], false) + err := r.state.Update(k, tmpKVEntries[k], false) if err != nil { return err } if value := tmpKVEntries[k]; value != nil { - t.kvEntries[k.String()] = string(value) + r.kvEntries[k.String()] = string(value) } else { - delete(t.kvEntries, k.String()) + delete(r.kvEntries, k.String()) } } return nil @@ -104,16 +106,16 @@ RetryLoop: } // MustApplyPatches calls ApplyPatches and must successfully -func (t *ReactorStateTester) MustApplyPatches() { - t.c.Assert(t.ApplyPatches(), check.IsNil) +func (r *ReactorStateTester) MustApplyPatches() { + require.Nil(r.t, r.ApplyPatches()) } // MustUpdate calls Update and must successfully -func (t *ReactorStateTester) MustUpdate(key string, value []byte) { - t.c.Assert(t.Update(key, value), check.IsNil) +func (r *ReactorStateTester) MustUpdate(key string, value []byte) { + require.Nil(r.t, r.Update(key, value)) } // KVEntries returns the contents of the mocked KV store. -func (t *ReactorStateTester) KVEntries() map[string]string { - return t.kvEntries +func (r *ReactorStateTester) KVEntries() map[string]string { + return r.kvEntries } diff --git a/pkg/p2p/client_test.go b/pkg/p2p/client_test.go index 345601fee16..471cf1bf74d 100644 --- a/pkg/p2p/client_test.go +++ b/pkg/p2p/client_test.go @@ -180,8 +180,7 @@ func TestMessageClientBasics(t *testing.T) { sender.AssertExpectations(t) // Test point 7: Interrupt the connection - grpcStream.ExpectedCalls = nil - grpcStream.Calls = nil + grpcStream.ResetMock() sender.ExpectedCalls = nil sender.Calls = nil diff --git a/pkg/p2p/mock_grpc_client.go b/pkg/p2p/mock_grpc_client.go index 7b32816875f..f9121e716d2 100644 --- a/pkg/p2p/mock_grpc_client.go +++ b/pkg/p2p/mock_grpc_client.go @@ -15,6 +15,7 @@ package p2p import ( "context" + "sync" "sync/atomic" "github.com/pingcap/tiflow/proto/p2p" @@ -22,8 +23,8 @@ import ( "google.golang.org/grpc" ) -//nolint:unused type mockSendMessageClient struct { + mu sync.Mutex mock.Mock // embeds an empty interface p2p.CDCPeerToPeer_SendMessageClient @@ -41,13 +42,24 @@ func newMockSendMessageClient(ctx context.Context) *mockSendMessageClient { } func (s *mockSendMessageClient) Send(packet *p2p.MessagePacket) error { + s.mu.Lock() + defer s.mu.Unlock() + args := s.Called(packet) atomic.AddInt32(&s.msgCount, 1) return args.Error(0) } func (s *mockSendMessageClient) Recv() (*p2p.SendMessageResponse, error) { - args := s.Called() + var args mock.Arguments + func() { + // We use a deferred Unlock in case `s.Called()` panics. + s.mu.Lock() + defer s.mu.Unlock() + + args = s.MethodCalled("Recv") + }() + if err := args.Error(1); err != nil { return nil, err } @@ -66,12 +78,18 @@ func (s *mockSendMessageClient) Context() context.Context { return s.ctx } -//nolint:unused +func (s *mockSendMessageClient) ResetMock() { + s.mu.Lock() + defer s.mu.Unlock() + + s.ExpectedCalls = nil + s.Calls = nil +} + type mockCDCPeerToPeerClient struct { mock.Mock } -//nolint:unused func (c *mockCDCPeerToPeerClient) SendMessage( ctx context.Context, opts ...grpc.CallOption, ) (p2p.CDCPeerToPeer_SendMessageClient, error) { diff --git a/pkg/util/cancel_monitor.go b/pkg/util/cancel_monitor.go index fb4953bc9cb..3e22ce4d288 100644 --- a/pkg/util/cancel_monitor.go +++ b/pkg/util/cancel_monitor.go @@ -21,12 +21,14 @@ import ( "go.uber.org/zap" ) -// MonitorCancelLatency monitors the latency from ctx being cancelled and the returned function being called -func MonitorCancelLatency(ctx context.Context, identifier string) func() { +// MonitorCancelLatency monitors the latency from ctx being cancelled +// the first returned function should be called when the cancellation is done +// the second returned function should be called to mark the cancellation is started, it will start a +// background go routine to monitor the latency util finish is called or cancellation is done +func MonitorCancelLatency(ctx context.Context, identifier string) (func(), func()) { finishedCh := make(chan struct{}) - go func() { - select { - case <-ctx.Done(): + start := func() { + go func() { log.Debug("MonitorCancelLatency: Cancelled", zap.String("identifier", identifier)) ticker := time.NewTicker(time.Second) defer ticker.Stop() @@ -43,10 +45,9 @@ func MonitorCancelLatency(ctx context.Context, identifier string) func() { zap.Int("duration", elapsed), zap.Error(ctx.Err())) } } - case <-finishedCh: - } - }() + }() + } return func() { close(finishedCh) - } + }, start } diff --git a/pkg/version/check.go b/pkg/version/check.go index ccc5d064fb0..6647bc10ec0 100644 --- a/pkg/version/check.go +++ b/pkg/version/check.go @@ -35,7 +35,6 @@ import ( var ( // minPDVersion is the version of the minimal compatible PD. - // TODO bump 5.2.0-alpha once PD releases. minPDVersion *semver.Version = semver.New("5.1.0-alpha") // maxPDVersion is the version of the maximum compatible PD. // Compatible versions are in [minPDVersion, maxPDVersion) @@ -43,16 +42,14 @@ var ( maxPDVersion *semver.Version = semver.New("9999.0.0") // MinTiKVVersion is the version of the minimal compatible TiKV. - // TODO bump 5.2.0-alpha once TiKV releases. MinTiKVVersion *semver.Version = semver.New("5.1.0-alpha") // maxTiKVVersion is the version of the maximum compatible TiKV. // Compatible versions are in [MinTiKVVersion, maxTiKVVersion) // 9999.0.0 disables the check effectively in the master branch. maxTiKVVersion *semver.Version = semver.New("9999.0.0") - // minTiCDCVersion is the version of the minimal compatible TiCDC. - // TODO bump 5.2.0-alpha once TiCDC releases. - minTiCDCVersion *semver.Version = semver.New("5.1.0-alpha") + // MinTiCDCVersion is the version of the minimal compatible TiCDC. + MinTiCDCVersion *semver.Version = semver.New("5.1.0-alpha") // Compatible versions are in [MinTiCDCVersion, MaxTiCDCVersion) // 9999.0.0 disables the check effectively in the master branch. maxTiCDCVersion *semver.Version = semver.New("9999.0.0") @@ -110,28 +107,29 @@ func CheckPDVersion(ctx context.Context, pdAddr string, credential *security.Cre req, err := http.NewRequestWithContext( ctx, http.MethodGet, fmt.Sprintf("%s/pd/api/v1/version", pdAddr), nil) if err != nil { - return cerror.WrapError(cerror.ErrCheckClusterVersionFromPD, err) + return cerror.ErrCheckClusterVersionFromPD.GenWithStackByArgs(err) } resp, err := httpClient.Do(req) if err != nil { - return cerror.WrapError(cerror.ErrCheckClusterVersionFromPD, err) + return cerror.ErrCheckClusterVersionFromPD.GenWithStackByArgs(err) } defer resp.Body.Close() - if resp.StatusCode < 200 || resp.StatusCode >= 300 { - arg := fmt.Sprintf("response status: %s", resp.Status) - return cerror.ErrCheckClusterVersionFromPD.GenWithStackByArgs(arg) - } - content, err := io.ReadAll(resp.Body) - if err != nil { - return cerror.WrapError(cerror.ErrCheckClusterVersionFromPD, err) + if err != nil || resp.StatusCode < 200 || resp.StatusCode >= 300 { + var arg string + if err != nil { + arg = fmt.Sprintf("%s %s %s", resp.Status, content, err) + } else { + arg = fmt.Sprintf("%s %s", resp.Status, content) + } + return cerror.ErrCheckClusterVersionFromPD.GenWithStackByArgs(arg) } err = json.Unmarshal(content, &pdVer) if err != nil { - return cerror.WrapError(cerror.ErrCheckClusterVersionFromPD, err) + return cerror.ErrCheckClusterVersionFromPD.GenWithStackByArgs(err) } ver, err := semver.NewVersion(removeVAndHash(pdVer.Version)) @@ -265,11 +263,11 @@ func CheckTiCDCClusterVersion(cdcClusterVer TiCDCClusterVersion) (unknown bool, return true, nil } ver := cdcClusterVer.Version - minOrd := ver.Compare(*minTiCDCVersion) + minOrd := ver.Compare(*MinTiCDCVersion) if minOrd < 0 { arg := fmt.Sprintf("TiCDC %s is not supported, the minimal compatible version is %s"+ "try tiup ctl:%s cdc [COMMAND]", - ver, minTiCDCVersion, ver) + ver, MinTiCDCVersion, ver) return false, cerror.ErrVersionIncompatible.GenWithStackByArgs(arg) } maxOrd := ver.Compare(*maxTiCDCVersion) diff --git a/pkg/version/check_test.go b/pkg/version/check_test.go index 38118c49ca2..feaae574ca3 100644 --- a/pkg/version/check_test.go +++ b/pkg/version/check_test.go @@ -17,6 +17,7 @@ import ( "context" "fmt" "net/http" + "net/http/httptest" "net/url" "testing" "time" @@ -171,7 +172,7 @@ func TestCheckClusterVersion(t *testing.T) { } err := CheckClusterVersion(context.Background(), &mock, pdAddrs, nil, false) - require.Regexp(t, ".*response status: .*", err) + require.Regexp(t, ".*400 Bad Request.*", err) } } @@ -337,7 +338,7 @@ func TestCheckTiCDCClusterVersion(t *testing.T) { expectedUnknown: true, }, { - cdcClusterVersion: TiCDCClusterVersion{Version: minTiCDCVersion}, + cdcClusterVersion: TiCDCClusterVersion{Version: MinTiCDCVersion}, expectedErr: "", expectedUnknown: false, }, @@ -361,3 +362,20 @@ func TestCheckTiCDCClusterVersion(t *testing.T) { } } } + +func TestCheckPDVersionError(t *testing.T) { + t.Parallel() + + var resp func(w http.ResponseWriter, r *http.Request) + ts := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + resp(w, r) + })) + defer ts.Close() + + resp = func(w http.ResponseWriter, _ *http.Request) { + w.WriteHeader(http.StatusInternalServerError) + } + require.Contains(t, CheckPDVersion(context.TODO(), ts.URL, nil).Error(), + "[CDC:ErrCheckClusterVersionFromPD]failed to request PD 500 Internal Server Error , please try again later", + ) +} diff --git a/scripts/canal/docker/Dockerfile b/scripts/canal/docker/Dockerfile index 9f1da0f986d..9dcedecf9c9 100644 --- a/scripts/canal/docker/Dockerfile +++ b/scripts/canal/docker/Dockerfile @@ -8,8 +8,9 @@ RUN yum clean all RUN yum makecache # Install mysql client. -RUN wget https://dev.mysql.com/get/mysql57-community-release-el6-9.noarch.rpm -RUN rpm -ivh mysql57-community-release-el6-9.noarch.rpm +RUN rpm -ivh https://repo.mysql.com/mysql57-community-release-el7-11.noarch.rpm +# See: https://support.cpanel.net/hc/en-us/articles/4419382481815?input_string=gpg+keys+problem+with+mysql+5.7 +RUN rpm --import https://repo.mysql.com/RPM-GPG-KEY-mysql-2022 RUN yum install mysql-community-client.x86_64 -y WORKDIR /root diff --git a/tests/integration_tests/_utils/start_tidb_cluster_impl b/tests/integration_tests/_utils/start_tidb_cluster_impl index 82d36de3a5a..6d121376b50 100755 --- a/tests/integration_tests/_utils/start_tidb_cluster_impl +++ b/tests/integration_tests/_utils/start_tidb_cluster_impl @@ -169,6 +169,10 @@ max-open-files = 4096 [raftstore] # true (default value) for high reliability, this can prevent data loss when power failure. sync-log = false +[cdc] +# tiflash proxy does not support this feature yet. +# TODO: enable hibernate-regions-compatible +hibernate-regions-compatible = false EOF # tidb server config file diff --git a/tools/check/check-errdoc.sh b/tools/check/check-errdoc.sh index 3ebf7269eca..5d77180a65f 100755 --- a/tools/check/check-errdoc.sh +++ b/tools/check/check-errdoc.sh @@ -17,5 +17,5 @@ set -euo pipefail cd -P . cp errors.toml /tmp/errors.toml.before -./tools/bin/errdoc-gen --source . --module github.com/pingcap/tiflow --output errors.toml --ignore proto,dm +./tools/bin/errdoc-gen --source . --module github.com/pingcap/tiflow --output errors.toml --ignore proto,dm,deployments diff -q errors.toml /tmp/errors.toml.before diff --git a/tools/check/go.sum b/tools/check/go.sum index ea23d874225..0903b3f2a69 100644 --- a/tools/check/go.sum +++ b/tools/check/go.sum @@ -296,6 +296,7 @@ github.com/golang/mock v1.4.1/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= github.com/golang/mock v1.5.0/go.mod h1:CWnOUgYIOo4TcNZ0wHX3YZCqsaM1I1Jvs6v3mP3KVu8= +github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc= github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= github.com/golang/protobuf v1.1.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=