Skip to content

Commit

Permalink
ddl: Move placement to special comment for ddl sql in binlog (#31454)
Browse files Browse the repository at this point in the history
close #31453
  • Loading branch information
lcwangchao authored Jan 11, 2022
1 parent aa7ad03 commit 50dfe6b
Show file tree
Hide file tree
Showing 4 changed files with 245 additions and 37 deletions.
128 changes: 99 additions & 29 deletions parser/ast/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ package ast

import (
"github.com/pingcap/errors"

"github.com/pingcap/tidb/parser/auth"
"github.com/pingcap/tidb/parser/format"
"github.com/pingcap/tidb/parser/model"
Expand Down Expand Up @@ -172,6 +171,16 @@ type AlterDatabaseStmt struct {

// Restore implements Node interface.
func (n *AlterDatabaseStmt) Restore(ctx *format.RestoreCtx) error {
// If all options placement options and RestoreTiDBSpecialComment flag is on,
// we should restore the whole node in special comment. For example, the restore result should be:
// /*T![placement] ALTER DATABASE `db1` PLACEMENT POLICY = `p1` */
// instead of
// ALTER DATABASE `db1` /*T![placement] PLACEMENT POLICY = `p1` */
// because altering a database without any options is not a legal syntax in mysql
if n.isAllPlacementOptions() && ctx.Flags.HasTiDBSpecialCommentFlag() {
return restorePlacementStmtInSpecialComment(ctx, n)
}

ctx.WriteKeyWord("ALTER DATABASE")
if !n.AlterDefaultDatabase {
ctx.WritePlain(" ")
Expand All @@ -197,6 +206,17 @@ func (n *AlterDatabaseStmt) Accept(v Visitor) (Node, bool) {
return v.Leave(n)
}

func (n *AlterDatabaseStmt) isAllPlacementOptions() bool {
for _, n := range n.Options {
switch n.Tp {
case DatabaseOptionPlacementPrimaryRegion, DatabaseOptionPlacementRegions, DatabaseOptionPlacementFollowerCount, DatabaseOptionPlacementLeaderConstraints, DatabaseOptionPlacementLearnerCount, DatabaseOptionPlacementVoterCount, DatabaseOptionPlacementSchedule, DatabaseOptionPlacementConstraints, DatabaseOptionPlacementFollowerConstraints, DatabaseOptionPlacementVoterConstraints, DatabaseOptionPlacementLearnerConstraints, DatabaseOptionPlacementPolicy:
default:
return false
}
}
return true
}

// DropDatabaseStmt is a statement to drop a database and all tables in the database.
// See https://dev.mysql.com/doc/refman/5.7/en/drop-database.html
type DropDatabaseStmt struct {
Expand Down Expand Up @@ -526,8 +546,9 @@ func (n *ColumnOption) Restore(ctx *format.RestoreCtx) error {
pkTp := n.PrimaryKeyTp.String()
if len(pkTp) != 0 {
ctx.WritePlain(" ")
ctx.WriteWithSpecialComments(tidb.FeatureIDClusteredIndex, func() {
_ = ctx.WriteWithSpecialComments(tidb.FeatureIDClusteredIndex, func() error {
ctx.WriteKeyWord(pkTp)
return nil
})
}
case ColumnOptionNotNull:
Expand Down Expand Up @@ -601,11 +622,12 @@ func (n *ColumnOption) Restore(ctx *format.RestoreCtx) error {
ctx.WriteKeyWord("STORAGE ")
ctx.WriteKeyWord(n.StrValue)
case ColumnOptionAutoRandom:
ctx.WriteWithSpecialComments(tidb.FeatureIDAutoRandom, func() {
_ = ctx.WriteWithSpecialComments(tidb.FeatureIDAutoRandom, func() error {
ctx.WriteKeyWord("AUTO_RANDOM")
if n.AutoRandomBitLength != types.UnspecifiedLength {
ctx.WritePlainf("(%d)", n.AutoRandomBitLength)
}
return nil
})
default:
return errors.New("An error occurred while splicing ColumnOption")
Expand Down Expand Up @@ -661,8 +683,9 @@ type IndexOption struct {
func (n *IndexOption) Restore(ctx *format.RestoreCtx) error {
hasPrevOption := false
if n.PrimaryKeyTp != model.PrimaryKeyTypeDefault {
ctx.WriteWithSpecialComments(tidb.FeatureIDClusteredIndex, func() {
_ = ctx.WriteWithSpecialComments(tidb.FeatureIDClusteredIndex, func() error {
ctx.WriteKeyWord(n.PrimaryKeyTp.String())
return nil
})
hasPrevOption = true
}
Expand Down Expand Up @@ -1199,6 +1222,10 @@ type DropPlacementPolicyStmt struct {

// Restore implements Restore interface.
func (n *DropPlacementPolicyStmt) Restore(ctx *format.RestoreCtx) error {
if ctx.Flags.HasTiDBSpecialCommentFlag() {
return restorePlacementStmtInSpecialComment(ctx, n)
}

ctx.WriteKeyWord("DROP PLACEMENT POLICY ")
if n.IfExists {
ctx.WriteKeyWord("IF EXISTS ")
Expand Down Expand Up @@ -1444,6 +1471,10 @@ type CreatePlacementPolicyStmt struct {

// Restore implements Node interface.
func (n *CreatePlacementPolicyStmt) Restore(ctx *format.RestoreCtx) error {
if ctx.Flags.HasTiDBSpecialCommentFlag() {
return restorePlacementStmtInSpecialComment(ctx, n)
}

ctx.WriteKeyWord("CREATE ")
if n.OrReplace {
ctx.WriteKeyWord("OR REPLACE ")
Expand Down Expand Up @@ -1690,8 +1721,9 @@ type DropIndexStmt struct {
func (n *DropIndexStmt) Restore(ctx *format.RestoreCtx) error {
ctx.WriteKeyWord("DROP INDEX ")
if n.IfExists {
ctx.WriteWithSpecialComments("", func() {
_ = ctx.WriteWithSpecialComments("", func() error {
ctx.WriteKeyWord("IF EXISTS ")
return nil
})
}
ctx.WriteName(n.IndexName)
Expand Down Expand Up @@ -1900,8 +1932,7 @@ type PlacementOption struct {
}

func (n *PlacementOption) Restore(ctx *format.RestoreCtx) error {
isSupported := true
fn := func() {
fn := func() error {
switch n.Tp {
case PlacementOptionPrimaryRegion:
ctx.WriteKeyWord("PRIMARY_REGION ")
Expand Down Expand Up @@ -1952,16 +1983,12 @@ func (n *PlacementOption) Restore(ctx *format.RestoreCtx) error {
ctx.WritePlain("= ")
ctx.WriteName(n.StrValue)
default:
isSupported = false
return errors.Errorf("invalid PlacementOption: %d", n.Tp)
}
}
if !isSupported {
return errors.Errorf("invalid PlacementOption: %d", n.Tp)
return nil
}
// WriteSpecialComment
ctx.WriteWithSpecialComments(tidb.FeatureIDPlacement, fn)

return nil
return ctx.WriteWithSpecialComments(tidb.FeatureIDPlacement, fn)
}

type StatsOptionType int
Expand Down Expand Up @@ -2110,31 +2137,35 @@ func (n *TableOption) Restore(ctx *format.RestoreCtx) error {
ctx.WriteKeyWord(n.StrValue)
case TableOptionAutoIncrement:
if n.BoolValue {
ctx.WriteWithSpecialComments(tidb.FeatureIDForceAutoInc, func() {
_ = ctx.WriteWithSpecialComments(tidb.FeatureIDForceAutoInc, func() error {
ctx.WriteKeyWord("FORCE")
return nil
})
ctx.WritePlain(" ")
}
ctx.WriteKeyWord("AUTO_INCREMENT ")
ctx.WritePlain("= ")
ctx.WritePlainf("%d", n.UintValue)
case TableOptionAutoIdCache:
ctx.WriteWithSpecialComments(tidb.FeatureIDAutoIDCache, func() {
_ = ctx.WriteWithSpecialComments(tidb.FeatureIDAutoIDCache, func() error {
ctx.WriteKeyWord("AUTO_ID_CACHE ")
ctx.WritePlain("= ")
ctx.WritePlainf("%d", n.UintValue)
return nil
})
case TableOptionAutoRandomBase:
if n.BoolValue {
ctx.WriteWithSpecialComments(tidb.FeatureIDForceAutoInc, func() {
_ = ctx.WriteWithSpecialComments(tidb.FeatureIDForceAutoInc, func() error {
ctx.WriteKeyWord("FORCE")
return nil
})
ctx.WritePlain(" ")
}
ctx.WriteWithSpecialComments(tidb.FeatureIDAutoRandomBase, func() {
_ = ctx.WriteWithSpecialComments(tidb.FeatureIDAutoRandomBase, func() error {
ctx.WriteKeyWord("AUTO_RANDOM_BASE ")
ctx.WritePlain("= ")
ctx.WritePlainf("%d", n.UintValue)
return nil
})
case TableOptionComment:
ctx.WriteKeyWord("COMMENT ")
Expand Down Expand Up @@ -2226,14 +2257,16 @@ func (n *TableOption) Restore(ctx *format.RestoreCtx) error {
ctx.WritePlainf("%d", n.UintValue)
}
case TableOptionShardRowID:
ctx.WriteWithSpecialComments(tidb.FeatureIDTiDB, func() {
_ = ctx.WriteWithSpecialComments(tidb.FeatureIDTiDB, func() error {
ctx.WriteKeyWord("SHARD_ROW_ID_BITS ")
ctx.WritePlainf("= %d", n.UintValue)
return nil
})
case TableOptionPreSplitRegion:
ctx.WriteWithSpecialComments(tidb.FeatureIDTiDB, func() {
_ = ctx.WriteWithSpecialComments(tidb.FeatureIDTiDB, func() error {
ctx.WriteKeyWord("PRE_SPLIT_REGIONS ")
ctx.WritePlainf("= %d", n.UintValue)
return nil
})
case TableOptionPackKeys:
// TODO: not support
Expand Down Expand Up @@ -2923,17 +2956,37 @@ func (n *AlterTableSpec) Restore(ctx *format.RestoreCtx) error {
ctx.WritePlainf("%d", n.Num)
}
case AlterTablePartitionOptions:
ctx.WriteKeyWord("PARTITION ")
ctx.WriteName(n.PartitionNames[0].O)
ctx.WritePlain(" ")
restoreWithoutSpecialComment := func() error {
origFlags := ctx.Flags
defer func() {
ctx.Flags = origFlags
}()
ctx.Flags &= ^format.RestoreTiDBSpecialComment
ctx.WriteKeyWord("PARTITION ")
ctx.WriteName(n.PartitionNames[0].O)
ctx.WritePlain(" ")

for i, opt := range n.Options {
if i != 0 {
ctx.WritePlain(" ")
}
if err := opt.Restore(ctx); err != nil {
return errors.Annotatef(err, "An error occurred while restore AlterTableSpec.Options[%d] for PARTITION `%s`", i, n.PartitionNames[0].O)
for i, opt := range n.Options {
if i != 0 {
ctx.WritePlain(" ")
}
if err := opt.Restore(ctx); err != nil {
return errors.Annotatef(err, "An error occurred while restore AlterTableSpec.Options[%d] for PARTITION `%s`", i, n.PartitionNames[0].O)
}
}
return nil
}

var err error
if ctx.Flags.HasTiDBSpecialCommentFlag() {
// AlterTablePartitionOptions now only supports placement options, so add put all options to special comment
err = ctx.WriteWithSpecialComments(tidb.FeatureIDPlacement, restoreWithoutSpecialComment)
} else {
err = restoreWithoutSpecialComment()
}

if err != nil {
return err
}
case AlterTablePartitionAttributes:
ctx.WriteKeyWord("PARTITION ")
Expand Down Expand Up @@ -3945,6 +3998,10 @@ type AlterPlacementPolicyStmt struct {
}

func (n *AlterPlacementPolicyStmt) Restore(ctx *format.RestoreCtx) error {
if ctx.Flags.HasTiDBSpecialCommentFlag() {
return restorePlacementStmtInSpecialComment(ctx, n)
}

ctx.WriteKeyWord("ALTER PLACEMENT POLICY ")
if n.IfExists {
ctx.WriteKeyWord("IF EXISTS ")
Expand Down Expand Up @@ -4009,3 +4066,16 @@ func (n *AlterSequenceStmt) Accept(v Visitor) (Node, bool) {
n.Name = node.(*TableName)
return v.Leave(n)
}

func restorePlacementStmtInSpecialComment(ctx *format.RestoreCtx, n DDLNode) error {
origFlags := ctx.Flags
defer func() {
ctx.Flags = origFlags
}()

ctx.Flags |= format.RestoreTiDBSpecialComment
return ctx.WriteWithSpecialComments(tidb.FeatureIDPlacement, func() error {
ctx.Flags &= ^format.RestoreTiDBSpecialComment
return n.Restore(ctx)
})
}
Loading

0 comments on commit 50dfe6b

Please sign in to comment.