diff --git a/tests/realtikvtest/addindextest/BUILD.bazel b/tests/realtikvtest/addindextest/BUILD.bazel index 0822f7288fc83..ac27f5bd58948 100644 --- a/tests/realtikvtest/addindextest/BUILD.bazel +++ b/tests/realtikvtest/addindextest/BUILD.bazel @@ -24,6 +24,7 @@ go_test( srcs = [ "add_index_test.go", "concurrent_ddl_test.go", + "failpoints_test.go", "main_test.go", "multi_schema_change_test.go", "pitr_test.go", diff --git a/tests/realtikvtest/addindextest/common.go b/tests/realtikvtest/addindextest/common.go index aaf29064f5760..d62edc11be5cd 100644 --- a/tests/realtikvtest/addindextest/common.go +++ b/tests/realtikvtest/addindextest/common.go @@ -19,7 +19,9 @@ import ( "strconv" "sync" "testing" + "time" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/logutil" @@ -33,19 +35,21 @@ const ( ) type suiteContext struct { - ctx context.Context - cancel func() - store kv.Storage - t *testing.T - tk *testkit.TestKit - isUnique bool - isPK bool - tableNum int - colNum int - rowNum int - workload *workload - tkPool *sync.Pool - CompCtx *CompatibilityContext + ctx context.Context + cancel func() + store kv.Storage + t *testing.T + tk *testkit.TestKit + isUnique bool + isPK bool + tableNum int + colNum int + rowNum int + workload *workload + tkPool *sync.Pool + isFailpointsTest bool + failSync sync.WaitGroup + CompCtx *CompatibilityContext } func (s *suiteContext) getTestKit() *testkit.TestKit { @@ -67,12 +71,13 @@ func (s *suiteContext) done() bool { func newSuiteContext(t *testing.T, tk *testkit.TestKit, store kv.Storage) *suiteContext { return &suiteContext{ - store: store, - t: t, - tk: tk, - tableNum: 3, - colNum: 28, - rowNum: 64, + store: store, + t: t, + tk: tk, + tableNum: 3, + colNum: 28, + rowNum: 64, + isFailpointsTest: false, } } @@ -346,6 +351,10 @@ func testOneColFrame(ctx *suiteContext, colIDs [][]int, f func(*suiteContext, in if ctx.workload != nil { ctx.workload.start(ctx, tableID, i) } + if ctx.isFailpointsTest { + ctx.failSync.Add(1) + go useFailpoints(ctx, i) + } err := f(ctx, tableID, tableName, i) if err != nil { if ctx.isUnique || ctx.isPK { @@ -358,6 +367,9 @@ func testOneColFrame(ctx *suiteContext, colIDs [][]int, f func(*suiteContext, in if ctx.workload != nil { _ = ctx.workload.stop(ctx, -1) } + if ctx.isFailpointsTest { + ctx.failSync.Wait() + } if err == nil { checkResult(ctx, tableName, i, tableID) } @@ -374,6 +386,10 @@ func testTwoColsFrame(ctx *suiteContext, iIDs [][]int, jIDs [][]int, f func(*sui if ctx.workload != nil { ctx.workload.start(ctx, tableID, i, j) } + if ctx.isFailpointsTest { + ctx.failSync.Add(1) + go useFailpoints(ctx, i) + } err := f(ctx, tableID, tableName, indexID, i, j) if err != nil { logutil.BgLogger().Error("[add index test] add index failed", zap.Error(err)) @@ -383,6 +399,9 @@ func testTwoColsFrame(ctx *suiteContext, iIDs [][]int, jIDs [][]int, f func(*sui // Stop workload _ = ctx.workload.stop(ctx, -1) } + if ctx.isFailpointsTest { + ctx.failSync.Wait() + } if err == nil && i != j { checkResult(ctx, tableName, indexID, tableID) } @@ -398,6 +417,10 @@ func testOneIndexFrame(ctx *suiteContext, colID int, f func(*suiteContext, int, if ctx.workload != nil { ctx.workload.start(ctx, tableID, colID) } + if ctx.isFailpointsTest { + ctx.failSync.Add(1) + go useFailpoints(ctx, tableID) + } err := f(ctx, tableID, tableName, colID) if err != nil { logutil.BgLogger().Error("[add index test] add index failed", zap.Error(err)) @@ -406,6 +429,9 @@ func testOneIndexFrame(ctx *suiteContext, colID int, f func(*suiteContext, int, if ctx.workload != nil { _ = ctx.workload.stop(ctx, -1) } + if ctx.isFailpointsTest { + ctx.failSync.Wait() + } if err == nil { if ctx.isPK { checkTableResult(ctx, tableName, tableID) @@ -472,3 +498,30 @@ func addIndexMultiCols(ctx *suiteContext, tableID int, tableName string, indexID } return err } + +type failpointsPath struct { + failpath string + inTerm string +} + +var failpoints = []failpointsPath{ + {"github.com/pingcap/tidb/ddl/EnablePiTR", "return"}, + {"github.com/pingcap/tidb/ddl/mockHighLoadForAddIndex", "return"}, + {"github.com/pingcap/tidb/ddl/mockBackfillRunErr", "1*return"}, + {"github.com/pingcap/tidb/ddl/mockBackfillSlow", "return"}, + {"github.com/pingcap/tidb/ddl/MockCaseWhenParseFailure", "return(true)"}, + {"github.com/pingcap/tidb/ddl/mockHighLoadForMergeIndex", "return"}, + {"github.com/pingcap/tidb/ddl/mockMergeRunErr", "1*return"}, + {"github.com/pingcap/tidb/ddl/mockMergeSlow", "return"}, +} + +func useFailpoints(ctx *suiteContext, failpos int) { + defer ctx.failSync.Done() + logutil.BgLogger().Info("stack", zap.Stack("cur stack"), zap.Int("id:", failpos)) + failpos %= 8 + require.NoError(ctx.t, failpoint.Enable(failpoints[failpos].failpath, failpoints[failpos].inTerm)) + logutil.BgLogger().Info("stack", zap.Stack("cur stack"), zap.Int("id:", failpos), zap.Bool("enable failpoints:", true)) + time.Sleep(10 * time.Second) + require.NoError(ctx.t, failpoint.Disable(failpoints[failpos].failpath)) + logutil.BgLogger().Info("stack", zap.Stack("cur stack"), zap.Int("id:", failpos), zap.Bool("disable failpoints:", true)) +} diff --git a/tests/realtikvtest/addindextest/failpoints_test.go b/tests/realtikvtest/addindextest/failpoints_test.go new file mode 100644 index 0000000000000..96d90231284f2 --- /dev/null +++ b/tests/realtikvtest/addindextest/failpoints_test.go @@ -0,0 +1,75 @@ +// 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package addindextest + +import ( + "testing" +) + +func initTestFailpoint(t *testing.T) *suiteContext { + ctx := initTest(t) + ctx.isFailpointsTest = true + return ctx +} + +func TestFailpointsCreateNonUniqueIndex(t *testing.T) { + t.Skip() + var colIDs = [][]int{ + {1, 4, 7, 10, 13, 16, 19, 22, 25}, + {2, 5, 8, 11, 14, 17, 20, 23, 26}, + {3, 6, 9, 12, 15, 18, 21, 24, 27}, + } + ctx := initTestFailpoint(t) + testOneColFrame(ctx, colIDs, addIndexNonUnique) +} + +func TestFailpointsCreateUniqueIndex(t *testing.T) { + t.Skip() + var colIDs = [][]int{ + {1, 6, 7, 8, 11, 13, 15, 16, 18, 19, 22, 26}, + {2, 9, 11, 17}, + {3, 12, 25}, + } + ctx := initTestFailpoint(t) + testOneColFrame(ctx, colIDs, addIndexUnique) +} + +func TestFailpointsCreatePrimaryKeyFailpoints(t *testing.T) { + t.Skip() + ctx := initTest(t) + testOneIndexFrame(ctx, 0, addIndexPK) +} + +func TestFailpointsCreateGenColIndex(t *testing.T) { + t.Skip() + ctx := initTestFailpoint(t) + testOneIndexFrame(ctx, 29, addIndexGenCol) +} + +func TestFailpointsCreateMultiColsIndex(t *testing.T) { + t.Skip() + var coliIDs = [][]int{ + {1, 4, 7}, + {2, 5, 8}, + {3, 6, 9}, + } + var coljIDs = [][]int{ + {16, 19, 22}, + {14, 17, 20}, + {18, 21, 24}, + } + ctx := initTestFailpoint(t) + testTwoColsFrame(ctx, coliIDs, coljIDs, addIndexMultiCols) +}