Skip to content

Commit

Permalink
ddl: forbid dropping placement policy which is in use (#28057)
Browse files Browse the repository at this point in the history
  • Loading branch information
lcwangchao authored Sep 17, 2021
1 parent 89a5e1b commit 8d66cb4
Show file tree
Hide file tree
Showing 9 changed files with 225 additions and 2 deletions.
4 changes: 4 additions & 0 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -6381,6 +6381,10 @@ func (d *ddl) DropPlacementPolicy(ctx sessionctx.Context, stmt *ast.DropPlacemen
return err
}

if err = checkPlacementPolicyNotInUseFromInfoSchema(is, policy); err != nil {
return err
}

job := &model.Job{
SchemaID: policy.ID,
SchemaName: policy.Name.L,
Expand Down
2 changes: 1 addition & 1 deletion ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -829,7 +829,7 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64,
case model.ActionCreatePlacementPolicy:
ver, err = onCreatePlacementPolicy(d, t, job)
case model.ActionDropPlacementPolicy:
ver, err = onDropPlacementPolicy(t, job)
ver, err = onDropPlacementPolicy(d, t, job)
case model.ActionAlterPlacementPolicy:
ver, err = onAlterPlacementPolicy(t, job)
default:
Expand Down
3 changes: 3 additions & 0 deletions ddl/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -262,6 +262,9 @@ var (
// ErrPlacementPolicyWithDirectOption is returned when create/alter table with both placement policy and placement options existed.
ErrPlacementPolicyWithDirectOption = dbterror.ClassDDL.NewStd(mysql.ErrPlacementPolicyWithDirectOption)

// ErrPlacementPolicyInUse is returned when placement policy is in use in drop/alter.
ErrPlacementPolicyInUse = dbterror.ClassDDL.NewStd(mysql.ErrPlacementPolicyInUse)

// ErrMultipleDefConstInListPart returns multiple definition of same constant in list partitioning.
ErrMultipleDefConstInListPart = dbterror.ClassDDL.NewStd(mysql.ErrMultipleDefConstInListPart)

Expand Down
61 changes: 60 additions & 1 deletion ddl/placement_policy.go
Original file line number Diff line number Diff line change
Expand Up @@ -144,11 +144,20 @@ func checkPlacementPolicyExistAndCancelNonExistJob(t *meta.Meta, job *model.Job,
return nil, err
}

func onDropPlacementPolicy(t *meta.Meta, job *model.Job) (ver int64, _ error) {
func onDropPlacementPolicy(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) {
policyInfo, err := checkPlacementPolicyExistAndCancelNonExistJob(t, job, job.SchemaID)
if err != nil {
return ver, errors.Trace(err)
}

err = checkPlacementPolicyNotInUse(d, t, policyInfo)
if err != nil {
if ErrPlacementPolicyInUse.Equal(err) {
job.State = model.JobStateCancelled
}
return ver, errors.Trace(err)
}

switch policyInfo.State {
case model.StatePublic:
// public -> write only
Expand Down Expand Up @@ -232,3 +241,53 @@ func onAlterPlacementPolicy(t *meta.Meta, job *model.Job) (ver int64, _ error) {
job.FinishDBJob(model.JobStateDone, model.StatePublic, ver, nil)
return ver, nil
}

func checkPlacementPolicyNotInUse(d *ddlCtx, t *meta.Meta, policy *model.PolicyInfo) error {
currVer, err := t.GetSchemaVersion()
if err != nil {
return err
}
is := d.infoCache.GetLatest()
if is.SchemaMetaVersion() == currVer {
return checkPlacementPolicyNotInUseFromInfoSchema(is, policy)
}

return checkPlacementPolicyNotInUseFromMeta(t, policy)
}

func checkPlacementPolicyNotInUseFromInfoSchema(is infoschema.InfoSchema, policy *model.PolicyInfo) error {
for _, dbInfo := range is.AllSchemas() {
// TODO: check policy is not in use for databases
for _, tbl := range is.SchemaTables(dbInfo.Name) {
tblInfo := tbl.Meta()
if ref := tblInfo.PlacementPolicyRef; ref != nil && ref.ID == policy.ID {
return ErrPlacementPolicyInUse.GenWithStackByArgs(policy.Name)
}
// TODO: check policy is not in use for partitions
}
}
return nil
}

func checkPlacementPolicyNotInUseFromMeta(t *meta.Meta, policy *model.PolicyInfo) error {
schemas, err := t.ListDatabases()
if err != nil {
return err
}

for _, dbInfo := range schemas {
// TODO: check policy is not in use for databases
tables, err := t.ListTables(dbInfo.ID)
if err != nil {
return err
}

for _, tblInfo := range tables {
if ref := tblInfo.PlacementPolicyRef; ref != nil && ref.ID == policy.ID {
return ErrPlacementPolicyInUse.GenWithStackByArgs(policy.Name)
}
// TODO: check policy is not in use for partitions
}
}
return nil
}
98 changes: 98 additions & 0 deletions ddl/placement_policy_ddl_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,98 @@
// 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,
// 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 ddl

import (
"context"

. "github.com/pingcap/check"
"github.com/pingcap/parser/model"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/sessionctx"
)

func testPlacementPolicyInfo(c *C, d *ddl, name string, settings *model.PlacementSettings) *model.PolicyInfo {
policy := &model.PolicyInfo{
Name: model.NewCIStr(name),
PlacementSettings: settings,
}
genIDs, err := d.genGlobalIDs(1)
c.Assert(err, IsNil)
policy.ID = genIDs[0]
return policy
}

func testCreatePlacementPolicy(c *C, ctx sessionctx.Context, d *ddl, policyInfo *model.PolicyInfo) *model.Job {
job := &model.Job{
SchemaName: policyInfo.Name.L,
Type: model.ActionCreatePlacementPolicy,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{policyInfo},
}
err := d.doDDLJob(ctx, job)
c.Assert(err, IsNil)

v := getSchemaVer(c, ctx)
policyInfo.State = model.StatePublic
checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v})
policyInfo.State = model.StateNone
return job
}

func (s *testDDLSuite) TestPlacementPolicyInUse(c *C) {
store := testCreateStore(c, "test_placement_policy_in_use")
defer func() {
err := store.Close()
c.Assert(err, IsNil)
}()

ctx := context.Background()
d := testNewDDLAndStart(ctx, c, WithStore(store))
sctx := testNewContext(d)

db1 := testSchemaInfo(c, d, "db1")
testCreateSchema(c, sctx, d, db1)
db2 := testSchemaInfo(c, d, "db2")
testCreateSchema(c, sctx, d, db2)

policySettings := &model.PlacementSettings{PrimaryRegion: "r1", Regions: "r1,r2"}
p1 := testPlacementPolicyInfo(c, d, "p1", policySettings)
p2 := testPlacementPolicyInfo(c, d, "p2", policySettings)
p3 := testPlacementPolicyInfo(c, d, "p3", policySettings)
testCreatePlacementPolicy(c, sctx, d, p1)
testCreatePlacementPolicy(c, sctx, d, p2)
testCreatePlacementPolicy(c, sctx, d, p3)

t1 := testTableInfo(c, d, "t1", 1)
t1.PlacementPolicyRef = &model.PolicyRefInfo{ID: p1.ID, Name: p1.Name}
testCreateTable(c, sctx, d, db1, t1)

t2 := testTableInfo(c, d, "t2", 1)
t2.PlacementPolicyRef = &model.PolicyRefInfo{ID: p1.ID, Name: p1.Name}
testCreateTable(c, sctx, d, db2, t2)

t3 := testTableInfo(c, d, "t3", 1)
t3.PlacementPolicyRef = &model.PolicyRefInfo{ID: p2.ID, Name: p2.Name}
testCreateTable(c, sctx, d, db1, t3)

c.Assert(kv.RunInNewTxn(ctx, sctx.GetStore(), false, func(ctx context.Context, txn kv.Transaction) error {
m := meta.NewMeta(txn)

c.Assert(ErrPlacementPolicyInUse.Equal(checkPlacementPolicyNotInUseFromMeta(m, p1)), IsTrue)
c.Assert(ErrPlacementPolicyInUse.Equal(checkPlacementPolicyNotInUseFromMeta(m, p2)), IsTrue)
c.Assert(checkPlacementPolicyNotInUseFromMeta(m, p3), IsNil)
return nil
}), IsNil)
}
52 changes: 52 additions & 0 deletions ddl/placement_policy_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -375,3 +375,55 @@ func (s *testDBSuite6) TestCreateTableWithPlacementPolicy(c *C) {
tk.MustExec("drop table if exists t")
tk.MustExec("drop placement policy if exists x")
}

func (s *testDBSuite6) TestDropPlacementPolicyInUse(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("create database if not exists test2")
tk.MustExec("drop table if exists test.t11, test.t12, test2.t21, test2.t21, test2.t22")
tk.MustExec("drop placement policy if exists p1")
tk.MustExec("drop placement policy if exists p2")
tk.MustExec("drop placement policy if exists p3")

// p1 is used by test.t11 and test2.t21
tk.MustExec("create placement policy p1 " +
"PRIMARY_REGION=\"cn-east-1\" " +
"REGIONS=\"cn-east-1, cn-east-2\" " +
"SCHEDULE=\"EVEN\"")
defer tk.MustExec("drop placement policy if exists p1")
tk.MustExec("create table test.t11 (id int) placement policy 'p1'")
defer tk.MustExec("drop table if exists test.t11")
tk.MustExec("create table test2.t21 (id int) placement policy 'p1'")
defer tk.MustExec("drop table if exists test2.t21")

// p1 is used by test.t12
tk.MustExec("create placement policy p2 " +
"PRIMARY_REGION=\"cn-east-1\" " +
"REGIONS=\"cn-east-1, cn-east-2\" " +
"SCHEDULE=\"EVEN\"")
defer tk.MustExec("drop placement policy if exists p2")
tk.MustExec("create table test.t12 (id int) placement policy 'p2'")
defer tk.MustExec("drop table if exists test.t12")

// p1 is used by test2.t22
tk.MustExec("create placement policy p3 " +
"PRIMARY_REGION=\"cn-east-1\" " +
"REGIONS=\"cn-east-1, cn-east-2\" " +
"SCHEDULE=\"EVEN\"")
defer tk.MustExec("drop placement policy if exists p3")
tk.MustExec("create table test.t21 (id int) placement policy 'p3'")
defer tk.MustExec("drop table if exists test.t21")

txn, err := s.store.Begin()
c.Assert(err, IsNil)
defer func() {
c.Assert(txn.Rollback(), IsNil)
}()
for _, policyName := range []string{"p1", "p2", "p3"} {
err := tk.ExecToErr(fmt.Sprintf("drop placement policy %s", policyName))
c.Assert(err.Error(), Equals, fmt.Sprintf("[ddl:8241]Placement policy '%s' is still in use", policyName))

err = tk.ExecToErr(fmt.Sprintf("drop placement policy if exists %s", policyName))
c.Assert(err.Error(), Equals, fmt.Sprintf("[ddl:8241]Placement policy '%s' is still in use", policyName))
}
}
1 change: 1 addition & 0 deletions errno/errcode.go
Original file line number Diff line number Diff line change
Expand Up @@ -1051,6 +1051,7 @@ const (
ErrPlacementPolicyExists = 8238
ErrPlacementPolicyNotExists = 8239
ErrPlacementPolicyWithDirectOption = 8240
ErrPlacementPolicyInUse = 8241

// TiKV/PD/TiFlash errors.
ErrPDServerTimeout = 9001
Expand Down
1 change: 1 addition & 0 deletions errno/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -1058,6 +1058,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{
ErrPlacementPolicyExists: mysql.Message("Placement policy '%-.192s' already exists", nil),
ErrPlacementPolicyNotExists: mysql.Message("Unknown placement policy '%-.192s'", nil),
ErrPlacementPolicyWithDirectOption: mysql.Message("Placement policy '%s' can't co-exist with direct placement options", nil),
ErrPlacementPolicyInUse: mysql.Message("Placement policy '%-.192s' is still in use", nil),

// TiKV/PD errors.
ErrPDServerTimeout: mysql.Message("PD server timeout", nil),
Expand Down
5 changes: 5 additions & 0 deletions errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -696,6 +696,11 @@ error = '''
Placement policy '%s' can't co-exist with direct placement options
'''

["ddl:8241"]
error = '''
Placement policy '%-.192s' is still in use
'''

["domain:8027"]
error = '''
Information schema is out of date: schema failed to update in 1 lease, please make sure TiDB can connect to TiKV
Expand Down

0 comments on commit 8d66cb4

Please sign in to comment.