Skip to content

Commit

Permalink
executor: sync deletable columns to binlog when remove record (#53617) (
Browse files Browse the repository at this point in the history
#54942)

close #53133
  • Loading branch information
ti-chi-bot authored Aug 5, 2024
1 parent 34a73ca commit fb67865
Show file tree
Hide file tree
Showing 6 changed files with 99 additions and 1 deletion.
2 changes: 2 additions & 0 deletions pkg/ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -171,6 +171,7 @@ func onAddColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error)
case model.StateWriteReorganization:
// reorganization -> public
// Adjust table column offset.
failpoint.InjectCall("onAddColumnStateWriteReorg")
offset, err := LocateOffsetToMove(columnInfo.Offset, pos, tblInfo)
if err != nil {
return ver, errors.Trace(err)
Expand Down Expand Up @@ -270,6 +271,7 @@ func onDropColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error)
}
case model.StateWriteOnly:
// write only -> delete only
failpoint.InjectCall("onDropColumnStateWriteOnly")
colInfo.State = model.StateDeleteOnly
tblInfo.MoveColumnInfo(colInfo.Offset, len(tblInfo.Columns)-1)
if len(idxInfos) > 0 {
Expand Down
1 change: 1 addition & 0 deletions pkg/executor/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -433,6 +433,7 @@ go_test(
"//pkg/testkit",
"//pkg/testkit/external",
"//pkg/testkit/testdata",
"//pkg/testkit/testfailpoint",
"//pkg/testkit/testmain",
"//pkg/testkit/testsetup",
"//pkg/types",
Expand Down
40 changes: 40 additions & 0 deletions pkg/executor/executor_txn_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,12 +18,15 @@ import (
"fmt"
"strconv"
"strings"
"sync"
"testing"
"time"

"github.com/pingcap/tidb/pkg/errno"
"github.com/pingcap/tidb/pkg/executor"
"github.com/pingcap/tidb/pkg/sessionctx/binloginfo"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/testkit/testfailpoint"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -694,3 +697,40 @@ func TestSavepointWithBinlog(t *testing.T) {
tk.MustExec("commit")
tk.MustQuery("select * from t").Check(testkit.Rows("1 1"))
}

func TestColumnNotMatchError(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.Session().GetSessionVars().BinlogClient = binloginfo.MockPumpsClient(&testkit.MockPumpClient{})
tk.MustExec("set @@global.tidb_enable_metadata_lock=0")
tk.MustExec("use test")
tk2 := testkit.NewTestKit(t, store)
tk2.MustExec("use test")
tk.MustExec("create table t(id int primary key, a int)")
tk.MustExec("insert into t values(1, 2)")

testfailpoint.EnableCall(t, "github.com/pingcap/tidb/pkg/ddl/onAddColumnStateWriteReorg", func() {
tk.MustExec("begin;")
})
var wg sync.WaitGroup
wg.Add(1)
go func() {
tk2.MustExec("alter table t add column wait_notify int")
wg.Done()
}()
wg.Wait()
tk.MustExec("delete from t where id=1")
tk.MustGetErrCode("commit", errno.ErrInfoSchemaChanged)

testfailpoint.EnableCall(t, "github.com/pingcap/tidb/pkg/ddl/onDropColumnStateWriteOnly", func() {
tk.MustExec("begin;")
})
wg.Add(1)
go func() {
tk2.MustExec("alter table t drop column wait_notify")
wg.Done()
}()
wg.Wait()
tk.MustExec("delete from t where id=1")
tk.MustGetErrCode("commit", errno.ErrInfoSchemaChanged)
}
2 changes: 1 addition & 1 deletion pkg/table/tables/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -1368,7 +1368,7 @@ func (t *TableCommon) RemoveRecord(ctx table.MutateContext, h kv.Handle, r []typ
memBuffer.Release(sh)

if shouldWriteBinlog(ctx.GetSessionVars(), t.meta) {
cols := t.Cols()
cols := t.DeletableCols()
colIDs := make([]int64, 0, len(cols)+1)
for _, col := range cols {
colIDs = append(colIDs, col.ID)
Expand Down
12 changes: 12 additions & 0 deletions pkg/testkit/testfailpoint/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,12 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library")

go_library(
name = "testfailpoint",
srcs = ["failpoint.go"],
importpath = "github.com/pingcap/tidb/pkg/testkit/testfailpoint",
visibility = ["//visibility:public"],
deps = [
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_stretchr_testify//require",
],
)
43 changes: 43 additions & 0 deletions pkg/testkit/testfailpoint/failpoint.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
// Copyright 2024 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 testfailpoint

import (
"testing"

"github.com/pingcap/failpoint"
"github.com/stretchr/testify/require"
)

// Enable enables fail-point, and disable it when test finished.
func Enable(t testing.TB, name, expr string) {
require.NoError(t, failpoint.Enable(name, expr))
t.Cleanup(func() {
require.NoError(t, failpoint.Disable(name))
})
}

// EnableCall enables fail-point, and disable it when test finished.
func EnableCall(t testing.TB, name string, fn any) {
require.NoError(t, failpoint.EnableCall(name, fn))
t.Cleanup(func() {
require.NoError(t, failpoint.Disable(name))
})
}

// Disable disables fail-point.
func Disable(t testing.TB, name string) {
require.NoError(t, failpoint.Disable(name))
}

0 comments on commit fb67865

Please sign in to comment.