Skip to content

Commit

Permalink
ddl: get latest old table ID before replace view (#53720) (#54576)
Browse files Browse the repository at this point in the history
close #53673
  • Loading branch information
ti-chi-bot authored Jul 22, 2024
1 parent f9e5b57 commit e0332bf
Show file tree
Hide file tree
Showing 8 changed files with 112 additions and 18 deletions.
2 changes: 1 addition & 1 deletion pkg/ddl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -149,7 +149,6 @@ go_library(
"//pkg/util/slice",
"//pkg/util/sqlexec",
"//pkg/util/stringutil",
"//pkg/util/syncutil",
"//pkg/util/timeutil",
"//pkg/util/topsql",
"//pkg/util/topsql/state",
Expand Down Expand Up @@ -315,6 +314,7 @@ go_test(
"@com_github_tikv_client_go_v2//util",
"@io_etcd_go_etcd_client_v3//:client",
"@org_golang_google_grpc//:grpc",
"@org_golang_x_sync//errgroup",
"@org_uber_go_atomic//:atomic",
"@org_uber_go_goleak//:goleak",
"@org_uber_go_zap//:zap",
Expand Down
6 changes: 0 additions & 6 deletions pkg/ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,6 @@ import (
"github.com/pingcap/tidb/pkg/util/dbterror"
"github.com/pingcap/tidb/pkg/util/gcutil"
"github.com/pingcap/tidb/pkg/util/logutil"
"github.com/pingcap/tidb/pkg/util/syncutil"
"github.com/tikv/client-go/v2/tikvrpc"
clientv3 "go.etcd.io/etcd/client/v3"
atomicutil "go.uber.org/atomic"
Expand Down Expand Up @@ -365,11 +364,6 @@ type ddlCtx struct {

// reorgCtx is used for reorganization.
reorgCtx reorgContexts
// backfillCtx is used for backfill workers.
backfillCtx struct {
syncutil.RWMutex
jobCtxMap map[int64]*JobContext
}

jobCtx struct {
sync.RWMutex
Expand Down
41 changes: 41 additions & 0 deletions pkg/ddl/ddl_api_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/util/chunk"
"github.com/stretchr/testify/require"
"golang.org/x/sync/errgroup"
)

func TestGetDDLJobs(t *testing.T) {
Expand Down Expand Up @@ -151,6 +152,46 @@ func enQueueDDLJobs(t *testing.T, sess session.Session, txn kv.Transaction, jobT
}
}

func TestCreateViewConcurrently(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")

tk.MustExec("create table t (a int);")
tk.MustExec("create view v as select * from t;")
var (
counterErr error
counter int
)
ddl.OnDDLCreateViewForTest = func(job *model.Job) {
counter++
if counter > 1 {
counterErr = fmt.Errorf("create view job should not run concurrently")
return
}
}
ddl.AfterDeliverToWorkerForTest = func(job *model.Job) {
if job.Type == model.ActionCreateView {
counter--
}
}
var eg errgroup.Group
for i := 0; i < 5; i++ {
eg.Go(func() error {
newTk := testkit.NewTestKit(t, store)
_, err := newTk.Exec("use test")
if err != nil {
return err
}
_, err = newTk.Exec("create or replace view v as select * from t;")
return err
})
}
err := eg.Wait()
require.NoError(t, err)
require.NoError(t, counterErr)
}

func TestCreateDropCreateTable(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
Expand Down
6 changes: 6 additions & 0 deletions pkg/ddl/job_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -356,13 +356,19 @@ func (d *ddl) loadDDLJobAndRun(se *sess.Session, pool *workerPool, getJob func(*
d.delivery2worker(wk, pool, job)
}

// AfterDeliverToWorkerForTest is only used for test.
var AfterDeliverToWorkerForTest func(*model.Job)

// delivery2worker owns the worker, need to put it back to the pool in this function.
func (d *ddl) delivery2worker(wk *worker, pool *workerPool, job *model.Job) {
injectFailPointForGetJob(job)
d.runningJobs.add(job)
d.wg.Run(func() {
metrics.DDLRunningJobCount.WithLabelValues(pool.tp().String()).Inc()
defer func() {
if AfterDeliverToWorkerForTest != nil {
AfterDeliverToWorkerForTest(job)
}
d.runningJobs.remove(job)
asyncNotify(d.ddlJobCh)
metrics.DDLRunningJobCount.WithLabelValues(pool.tp().String()).Dec()
Expand Down
4 changes: 2 additions & 2 deletions pkg/ddl/placement_policy.go
Original file line number Diff line number Diff line change
Expand Up @@ -119,7 +119,7 @@ func getPlacementPolicyByName(d *ddlCtx, t *meta.Meta, policyName model.CIStr) (
}

is := d.infoCache.GetLatest()
if is.SchemaMetaVersion() == currVer {
if is != nil && is.SchemaMetaVersion() == currVer {
// Use cached policy.
policy, ok := is.PolicyByName(policyName)
if ok {
Expand Down Expand Up @@ -346,7 +346,7 @@ func checkPlacementPolicyNotInUse(d *ddlCtx, t *meta.Meta, policy *model.PolicyI
return err
}
is := d.infoCache.GetLatest()
if is.SchemaMetaVersion() == currVer {
if is != nil && is.SchemaMetaVersion() == currVer {
err = CheckPlacementPolicyNotInUseFromInfoSchema(is, policy)
} else {
err = CheckPlacementPolicyNotInUseFromMeta(t, policy)
Expand Down
2 changes: 1 addition & 1 deletion pkg/ddl/schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ func checkSchemaNotExists(d *ddlCtx, t *meta.Meta, schemaID int64, dbInfo *model
return err
}
is := d.infoCache.GetLatest()
if is.SchemaMetaVersion() == currVer {
if is != nil && is.SchemaMetaVersion() == currVer {
return checkSchemaNotExistsFromInfoSchema(is, schemaID, dbInfo)
}
return checkSchemaNotExistsFromStore(t, schemaID, dbInfo)
Expand Down
66 changes: 59 additions & 7 deletions pkg/ddl/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -270,18 +270,28 @@ func repairTableOrViewWithCheck(t *meta.Meta, job *model.Job, schemaID int64, tb
return t.UpdateTable(schemaID, tbInfo)
}

// OnDDLCreateViewForTest is only used for test.
var OnDDLCreateViewForTest func(*model.Job)

func onCreateView(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) {
schemaID := job.SchemaID
tbInfo := &model.TableInfo{}
var orReplace bool
var oldTbInfoID int64
if err := job.DecodeArgs(tbInfo, &orReplace, &oldTbInfoID); err != nil {
var _placeholder int64 // oldTblInfoID
if err := job.DecodeArgs(tbInfo, &orReplace, &_placeholder); err != nil {
// Invalid arguments, cancel this job.
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}
tbInfo.State = model.StateNone
err := checkTableNotExists(d, t, schemaID, tbInfo.Name.L)

oldTableID, err := findTableIDByName(d, t, schemaID, tbInfo.Name.L)
if infoschema.ErrTableNotExists.Equal(err) {
err = nil
}
if OnDDLCreateViewForTest != nil {
OnDDLCreateViewForTest(job)
}
if err != nil {
if infoschema.ErrDatabaseNotExists.Equal(err) {
job.State = model.JobStateCancelled
Expand All @@ -303,13 +313,13 @@ func onCreateView(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error)
// none -> public
tbInfo.State = model.StatePublic
tbInfo.UpdateTS = t.StartTS
if oldTbInfoID > 0 && orReplace {
err = t.DropTableOrView(schemaID, oldTbInfoID)
if oldTableID > 0 && orReplace {
err = t.DropTableOrView(schemaID, oldTableID)
if err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}
err = t.GetAutoIDAccessors(schemaID, oldTbInfoID).Del()
err = t.GetAutoIDAccessors(schemaID, oldTableID).Del()
if err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
Expand Down Expand Up @@ -1474,7 +1484,7 @@ func checkTableNotExists(d *ddlCtx, t *meta.Meta, schemaID int64, tableName stri
return err
}
is := d.infoCache.GetLatest()
if is.SchemaMetaVersion() == currVer {
if is != nil && is.SchemaMetaVersion() == currVer {
return checkTableNotExistsFromInfoSchema(is, schemaID, tableName)
}

Expand Down Expand Up @@ -1527,6 +1537,48 @@ func checkTableNotExistsFromStore(t *meta.Meta, schemaID int64, tableName string
return nil
}

func findTableIDByName(d *ddlCtx, t *meta.Meta, schemaID int64, tableName string) (int64, error) {
// Try to use memory schema info to check first.
currVer, err := t.GetSchemaVersion()
if err != nil {
return 0, err
}
is := d.infoCache.GetLatest()
if is != nil && is.SchemaMetaVersion() == currVer {
return findTableIDFromInfoSchema(is, schemaID, tableName)
}

return findTableIDFromStore(t, schemaID, tableName)
}

func findTableIDFromInfoSchema(is infoschema.InfoSchema, schemaID int64, tableName string) (int64, error) {
schema, ok := is.SchemaByID(schemaID)
if !ok {
return 0, infoschema.ErrDatabaseNotExists.GenWithStackByArgs("")
}
tbl, err := is.TableByName(schema.Name, model.NewCIStr(tableName))
if err != nil {
return 0, err
}
return tbl.Meta().ID, nil
}

func findTableIDFromStore(t *meta.Meta, schemaID int64, tableName string) (int64, error) {
tbls, err := t.ListSimpleTables(schemaID)
if err != nil {
if meta.ErrDBNotExists.Equal(err) {
return 0, infoschema.ErrDatabaseNotExists.GenWithStackByArgs("")
}
return 0, errors.Trace(err)
}
for _, tbl := range tbls {
if tbl.Name.L == tableName {
return tbl.ID, nil
}
}
return 0, infoschema.ErrTableNotExists.FastGenByArgs(tableName)
}

// updateVersionAndTableInfoWithCheck checks table info validate and updates the schema version and the table information
func updateVersionAndTableInfoWithCheck(d *ddlCtx, t *meta.Meta, job *model.Job, tblInfo *model.TableInfo, shouldUpdateVer bool, multiInfos ...schemaIDAndTableInfo) (
ver int64, err error) {
Expand Down
3 changes: 2 additions & 1 deletion pkg/ddl/table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -309,7 +309,8 @@ func TestCreateView(t *testing.T) {
}
ctx.SetValue(sessionctx.QueryString, "skip")
err = d.DoDDLJob(ctx, job)
require.Error(t, err)
// The non-existing table id in job args will not be considered anymore.
require.NoError(t, err)
}

func checkTableCacheTest(t *testing.T, store kv.Storage, dbInfo *model.DBInfo, tblInfo *model.TableInfo) {
Expand Down

0 comments on commit e0332bf

Please sign in to comment.