Skip to content

Commit

Permalink
add delay to avoid breaking consistency for add-foreign-key and async…
Browse files Browse the repository at this point in the history
… commit

Signed-off-by: Yang Keao <yangkeao@chunibyo.icu>
  • Loading branch information
YangKeao committed Oct 8, 2024
1 parent 7daf026 commit 95f891b
Show file tree
Hide file tree
Showing 3 changed files with 68 additions and 0 deletions.
3 changes: 3 additions & 0 deletions pkg/ddl/foreign_key.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"strings"

"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/pkg/infoschema"
infoschemactx "github.com/pingcap/tidb/pkg/infoschema/context"
"github.com/pingcap/tidb/pkg/meta"
Expand Down Expand Up @@ -67,11 +68,13 @@ func (w *worker) onCreateForeignKey(jobCtx *jobContext, job *model.Job) (ver int
job.SchemaState = model.StateWriteOnly
return ver, nil
case model.StateWriteOnly:
delayForAsyncCommit()
err = checkForeignKeyConstrain(w, job.SchemaName, tblInfo.Name.L, fkInfo, fkCheck)
if err != nil {
job.State = model.JobStateRollingback
return ver, err
}
failpoint.InjectCall("afterCheckForeignKeyConstrain")
tblInfo.ForeignKeys[len(tblInfo.ForeignKeys)-1].State = model.StateWriteReorganization
ver, err = updateVersionAndTableInfo(jobCtx, job, tblInfo, true)
if err != nil {
Expand Down
2 changes: 2 additions & 0 deletions tests/realtikvtest/txntest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,9 @@ go_test(
"//pkg/session/txninfo",
"//pkg/store/driver",
"//pkg/testkit",
"//pkg/testkit/testfailpoint",
"//pkg/util",
"//pkg/util/sqlexec",
"//tests/realtikvtest",
"@com_github_pingcap_errors//:errors",
"@com_github_pingcap_failpoint//:failpoint",
Expand Down
63 changes: 63 additions & 0 deletions tests/realtikvtest/txntest/txn_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"context"
"fmt"
"strconv"
"sync"
"testing"
"time"

Expand All @@ -29,6 +30,8 @@ import (
"github.com/pingcap/tidb/pkg/expression"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/testkit/testfailpoint"
"github.com/pingcap/tidb/pkg/util/sqlexec"
"github.com/pingcap/tidb/tests/realtikvtest"
"github.com/stretchr/testify/require"
"github.com/tikv/client-go/v2/tikvrpc"
Expand Down Expand Up @@ -532,3 +535,63 @@ func TestCheckTxnStatusOnOptimisticTxnBreakConsistency(t *testing.T) {
tk2.MustExec("admin check table t2")
tk2.MustQuery("select * from t2 order by id").Check(testkit.Rows("1 10", "2 11"))
}

func TestDMLWithAddForeignKey(t *testing.T) {
defer config.RestoreFunc()()
config.UpdateGlobal(func(conf *config.Config) {
conf.TiKVClient.AsyncCommit.SafeWindow = 10 * time.Second
conf.TiKVClient.AsyncCommit.AllowedClockDrift = 500 * time.Millisecond
})

store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("set global tidb_enable_1pc='OFF';")
tk.MustExec("set global tidb_enable_metadata_lock='OFF';")
tk.MustExec("set global tidb_enable_async_commit='ON'")

tkDML := testkit.NewTestKit(t, store)
tkDML.MustExec("use test")

tkDDL := testkit.NewTestKit(t, store)
tkDDL.MustExec("use test")
tkDDL.MustExec("create table parent (id int primary key, val int, index(val));")
tkDDL.MustExec("create table child (id int primary key, val int, index(val));")

// The fail path of this test is:
// tk: INSERT -> ... -> Wait -> PreWrite -> ... -> Async Commit -> Wait -> ... -> Success.
// tkDDL: DDL -> StateWriteOnly -> checkForeignKeyConstrain -> DDL -> Success
// After fixing, either the `tkDDL` or `tk` will fail.
testfailpoint.Enable(t, "tikvclient/beforePrewrite", "pause")
testfailpoint.EnableCall(t, "github.com/pingcap/tidb/pkg/ddl/afterCheckForeignKeyConstrain", func() {
require.NoError(t, failpoint.Disable("tikvclient/beforePrewrite"))
})
testfailpoint.Enable(t, "tikvclient/asyncCommitDoNothing", "pause")

var wg sync.WaitGroup
var errDML, errDDL error
wg.Add(2)
go func() {
defer wg.Done()

var rs sqlexec.RecordSet
rs, errDML = tkDML.Exec("insert into child values (1, 1)")
if rs != nil {
rs.Close()
}
}()

go func() {
defer wg.Done()

var rs sqlexec.RecordSet
rs, errDDL = tkDDL.Exec("alter table child add foreign key fk(val) references parent (val);")
if rs != nil {
rs.Close()
}
require.NoError(t, failpoint.Disable("tikvclient/asyncCommitDoNothing"))
}()

wg.Wait()

require.True(t, errDML != nil || errDDL != nil)
}

0 comments on commit 95f891b

Please sign in to comment.