Skip to content

Commit

Permalink
*: allow setting placement policy and tiflash replica at the same tim…
Browse files Browse the repository at this point in the history
…e in one table (#37252)

close #37171
  • Loading branch information
lcwangchao authored Aug 23, 2022
1 parent cf69205 commit 75e0885
Show file tree
Hide file tree
Showing 5 changed files with 65 additions and 52 deletions.
4 changes: 0 additions & 4 deletions ddl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -243,10 +243,6 @@ func alterTablePartitionBundles(t *meta.Meta, tblInfo *model.TableInfo, addingDe
p.Definitions = append(tblInfo.Partition.Definitions, addingDefinitions...)
tblInfo.Partition = &p

if tblInfo.TiFlashReplica != nil && tblInfo.TiFlashReplica.Count > 0 && tableHasPlacementSettings(tblInfo) {
return nil, errors.Trace(dbterror.ErrIncompatibleTiFlashAndPlacement)
}

// bundle for table should be recomputed because it includes some default configs for partitions
tblBundle, err := placement.NewTableBundle(t, tblInfo)
if err != nil {
Expand Down
16 changes: 0 additions & 16 deletions ddl/placement_policy.go
Original file line number Diff line number Diff line change
Expand Up @@ -417,19 +417,3 @@ func checkPlacementPolicyNotUsedByTable(tblInfo *model.TableInfo, policy *model.

return nil
}

func tableHasPlacementSettings(tblInfo *model.TableInfo) bool {
if tblInfo.PlacementPolicyRef != nil {
return true
}

if tblInfo.Partition != nil {
for _, def := range tblInfo.Partition.Definitions {
if def.PlacementPolicyRef != nil {
return true
}
}
}

return false
}
80 changes: 65 additions & 15 deletions ddl/placement_sql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,11 +21,11 @@ import (
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/ddl"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/domain/infosync"
mysql "github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/util/dbterror"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -578,48 +578,89 @@ func TestPlacementMode(t *testing.T) {
tk.MustQuery("show create database db2").Check(testkit.Rows("db2 CREATE DATABASE `db2` /*!40100 DEFAULT CHARACTER SET utf8mb4 */"))
}

func checkTiflashReplicaSet(t *testing.T, do *domain.Domain, db, tb string, cnt uint64) {
tbl, err := do.InfoSchema().TableByName(model.NewCIStr(db), model.NewCIStr(tb))
require.NoError(t, err)

tiflashReplica := tbl.Meta().TiFlashReplica
if cnt == 0 {
require.Nil(t, tiflashReplica)
return
}

CheckPlacementRule(infosync.GetMockTiFlash(), *infosync.MakeNewRule(tbl.Meta().ID, 1, nil))
require.NotNil(t, tiflashReplica)
require.Equal(t, cnt, tiflashReplica.Count)
}

func TestPlacementTiflashCheck(t *testing.T) {
store := testkit.CreateMockStore(t)
store, do := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`))
defer func() {
err := failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount")
require.NoError(t, err)
}()

tiflash := infosync.NewMockTiFlash()
infosync.SetMockTiFlash(tiflash)
defer func() {
tiflash.Lock()
tiflash.StatusServer.Close()
tiflash.Unlock()
}()

tk.MustExec("use test")
tk.MustExec("drop placement policy if exists p1")
tk.MustExec("drop placement policy if exists p2")
tk.MustExec("drop table if exists tp")

tk.MustExec("create placement policy p1 primary_region='r1' regions='r1'")
defer tk.MustExec("drop placement policy if exists p1")

tk.MustExec("create placement policy p2 primary_region='r2' regions='r1,r2'")
defer tk.MustExec("drop placement policy if exists p2")

tk.MustExec(`CREATE TABLE tp (id INT) PARTITION BY RANGE (id) (
PARTITION p0 VALUES LESS THAN (100),
PARTITION p1 VALUES LESS THAN (1000)
)`)
defer tk.MustExec("drop table if exists tp")
checkTiflashReplicaSet(t, do, "test", "tp", 0)
tk.MustExec("alter table tp set tiflash replica 1")

err := tk.ExecToErr("alter table tp placement policy p1")
require.True(t, dbterror.ErrIncompatibleTiFlashAndPlacement.Equal(err))
err = tk.ExecToErr("alter table tp partition p0 placement policy p1")
require.True(t, dbterror.ErrIncompatibleTiFlashAndPlacement.Equal(err))
tk.MustExec("alter table tp placement policy p1")
checkExistTableBundlesInPD(t, do, "test", "tp")
checkTiflashReplicaSet(t, do, "test", "tp", 1)
tk.MustQuery("show create table tp").Check(testkit.Rows("" +
"tp CREATE TABLE `tp` (\n" +
" `id` int(11) DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PLACEMENT POLICY=`p1` */\n" +
"PARTITION BY RANGE (`id`)\n" +
"(PARTITION `p0` VALUES LESS THAN (100),\n" +
" PARTITION `p1` VALUES LESS THAN (1000))"))

tk.MustExec("alter table tp partition p0 placement policy p2")
checkExistTableBundlesInPD(t, do, "test", "tp")
checkTiflashReplicaSet(t, do, "test", "tp", 1)
tk.MustQuery("show create table tp").Check(testkit.Rows("" +
"tp CREATE TABLE `tp` (\n" +
" `id` int(11) DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PLACEMENT POLICY=`p1` */\n" +
"PARTITION BY RANGE (`id`)\n" +
"(PARTITION `p0` VALUES LESS THAN (100) /*T![placement] PLACEMENT POLICY=`p2` */,\n" +
" PARTITION `p1` VALUES LESS THAN (1000))"))

tk.MustExec("drop table tp")
tk.MustExec(`CREATE TABLE tp (id INT) placement policy p1 PARTITION BY RANGE (id) (
PARTITION p0 VALUES LESS THAN (100),
PARTITION p1 VALUES LESS THAN (1000)
)`)
err = tk.ExecToErr("alter table tp set tiflash replica 1")
require.True(t, dbterror.ErrIncompatibleTiFlashAndPlacement.Equal(err))
checkTiflashReplicaSet(t, do, "test", "tp", 0)

tk.MustExec("alter table tp set tiflash replica 1")
checkTiflashReplicaSet(t, do, "test", "tp", 1)
checkExistTableBundlesInPD(t, do, "test", "tp")
tk.MustQuery("show create table tp").Check(testkit.Rows("" +
"tp CREATE TABLE `tp` (\n" +
" `id` int(11) DEFAULT NULL\n" +
Expand All @@ -633,8 +674,11 @@ func TestPlacementTiflashCheck(t *testing.T) {
PARTITION p0 VALUES LESS THAN (100) placement policy p1 ,
PARTITION p1 VALUES LESS THAN (1000)
)`)
err = tk.ExecToErr("alter table tp set tiflash replica 1")
require.True(t, dbterror.ErrIncompatibleTiFlashAndPlacement.Equal(err))
checkTiflashReplicaSet(t, do, "test", "tp", 0)

tk.MustExec("alter table tp set tiflash replica 1")
checkExistTableBundlesInPD(t, do, "test", "tp")
checkTiflashReplicaSet(t, do, "test", "tp", 1)
tk.MustQuery("show create table tp").Check(testkit.Rows("" +
"tp CREATE TABLE `tp` (\n" +
" `id` int(11) DEFAULT NULL\n" +
Expand All @@ -648,8 +692,11 @@ func TestPlacementTiflashCheck(t *testing.T) {
PARTITION p0 VALUES LESS THAN (100),
PARTITION p1 VALUES LESS THAN (1000)
)`)
err = tk.ExecToErr("alter table tp set tiflash replica 1")
require.True(t, dbterror.ErrIncompatibleTiFlashAndPlacement.Equal(err))
checkTiflashReplicaSet(t, do, "test", "tp", 0)

tk.MustExec("alter table tp set tiflash replica 1")
checkExistTableBundlesInPD(t, do, "test", "tp")
checkTiflashReplicaSet(t, do, "test", "tp", 1)
tk.MustQuery("show create table tp").Check(testkit.Rows("" +
"tp CREATE TABLE `tp` (\n" +
" `id` int(11) DEFAULT NULL\n" +
Expand All @@ -663,8 +710,11 @@ func TestPlacementTiflashCheck(t *testing.T) {
PARTITION p0 VALUES LESS THAN (100) PLACEMENT POLICY p1,
PARTITION p1 VALUES LESS THAN (1000)
)`)
err = tk.ExecToErr("alter table tp set tiflash replica 1")
require.True(t, dbterror.ErrIncompatibleTiFlashAndPlacement.Equal(err))
checkTiflashReplicaSet(t, do, "test", "tp", 0)

tk.MustExec("alter table tp set tiflash replica 1")
checkExistTableBundlesInPD(t, do, "test", "tp")
checkTiflashReplicaSet(t, do, "test", "tp", 1)
tk.MustQuery("show create table tp").Check(testkit.Rows("" +
"tp CREATE TABLE `tp` (\n" +
" `id` int(11) DEFAULT NULL\n" +
Expand Down
15 changes: 0 additions & 15 deletions ddl/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -1108,11 +1108,6 @@ func (w *worker) onSetTableFlashReplica(d *ddlCtx, t *meta.Meta, job *model.Job)
return ver, errors.Trace(err)
}

if replicaInfo.Count > 0 && tableHasPlacementSettings(tblInfo) {
job.State = model.JobStateCancelled
return ver, errors.Trace(dbterror.ErrIncompatibleTiFlashAndPlacement)
}

// Ban setting replica count for tables in system database.
if tidb_util.IsMemOrSysDB(job.SchemaName) {
return ver, errors.Trace(dbterror.ErrUnsupportedTiFlashOperationForSysOrMemTable)
Expand Down Expand Up @@ -1473,11 +1468,6 @@ func onAlterTablePartitionPlacement(d *ddlCtx, t *meta.Meta, job *model.Job) (ve
return 0, err
}

if tblInfo.TiFlashReplica != nil && tblInfo.TiFlashReplica.Count > 0 {
job.State = model.JobStateCancelled
return 0, errors.Trace(dbterror.ErrIncompatibleTiFlashAndPlacement)
}

ptInfo := tblInfo.GetPartitionInfo()
var partitionDef *model.PartitionDefinition
definitions := ptInfo.Definitions
Expand Down Expand Up @@ -1543,11 +1533,6 @@ func onAlterTablePlacement(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64,
return 0, err
}

if tblInfo.TiFlashReplica != nil && tblInfo.TiFlashReplica.Count > 0 {
job.State = model.JobStateCancelled
return 0, errors.Trace(dbterror.ErrIncompatibleTiFlashAndPlacement)
}

if _, err = checkPlacementPolicyRefValidAndCanNonValidJob(t, job, policyRefInfo); err != nil {
return 0, errors.Trace(err)
}
Expand Down
2 changes: 0 additions & 2 deletions util/dbterror/ddl_terror.go
Original file line number Diff line number Diff line change
Expand Up @@ -367,8 +367,6 @@ var (
ErrDependentByFunctionalIndex = ClassDDL.NewStd(mysql.ErrDependentByFunctionalIndex)
// ErrFunctionalIndexOnBlob when the expression of expression index returns blob or text.
ErrFunctionalIndexOnBlob = ClassDDL.NewStd(mysql.ErrFunctionalIndexOnBlob)
// ErrIncompatibleTiFlashAndPlacement when placement and tiflash replica options are set at the same time
ErrIncompatibleTiFlashAndPlacement = ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("Placement and tiflash replica options cannot be set at the same time", nil))

// ErrUnsupportedAlterTableSpec means we don't support this alter table specification (i.e. unknown)
ErrUnsupportedAlterTableSpec = ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message(fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation].Raw, "Unsupported/unknown ALTER TABLE specification"), nil))
Expand Down

0 comments on commit 75e0885

Please sign in to comment.