Skip to content

Commit

Permalink
ddl: support alter table .. truncate partition (pingcap#8624)
Browse files Browse the repository at this point in the history
  • Loading branch information
tiancaiamao authored and iamzhoug37 committed Dec 13, 2018
1 parent bca4990 commit 8c9e4be
Show file tree
Hide file tree
Showing 9 changed files with 163 additions and 6 deletions.
1 change: 0 additions & 1 deletion .travis.yml
Original file line number Diff line number Diff line change
Expand Up @@ -17,4 +17,3 @@ before_install:
script:
- make dev
- ./hack/check-tidy.sh
- ./hack/check_parser_replace.sh
56 changes: 56 additions & 0 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3056,6 +3056,62 @@ func (s *testDBSuite) TestAlterTableDropPartition(c *C) {
s.testErrorCode(c, sql4, tmysql.ErrDropPartitionNonExistent)
}

func (s *testDBSuite) TestAlterTableTruncatePartition(c *C) {
s.tk = testkit.NewTestKit(c, s.store)
s.tk.MustExec("use test")
s.tk.MustExec("drop table if exists employees")
s.tk.MustExec("set @@tidb_enable_table_partition = 1")
s.tk.MustExec(`create table employees (
id int not null,
hired int not null
) partition by range( hired ) (
partition p1 values less than (1991),
partition p2 values less than (1996),
partition p3 values less than (2001)
)`)
s.tk.MustExec("insert into employees values (1, 1990)")
s.tk.MustExec("insert into employees values (2, 1995)")
s.tk.MustExec("insert into employees values (3, 2000)")
result := s.tk.MustQuery("select * from employees order by id")
result.Check(testkit.Rows(`1 1990`, `2 1995`, `3 2000`))

s.testErrorCode(c, "alter table employees truncate partition xxx", tmysql.ErrUnknownPartition)

ctx := s.tk.Se.(sessionctx.Context)
is := domain.GetDomain(ctx).InfoSchema()
oldTblInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("employees"))
c.Assert(err, IsNil)
oldPID := oldTblInfo.Meta().Partition.Definitions[0].ID

s.tk.MustExec("alter table employees truncate partition p1")
result = s.tk.MustQuery("select * from employees order by id")
result.Check(testkit.Rows(`2 1995`, `3 2000`))

partitionPrefix := tablecodec.EncodeTablePrefix(oldPID)
hasOldPartitionData := checkPartitionDelRangeDone(c, s, partitionPrefix)
c.Assert(hasOldPartitionData, IsFalse)

is = domain.GetDomain(ctx).InfoSchema()
oldTblInfo, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("employees"))
c.Assert(err, IsNil)
newPID := oldTblInfo.Meta().Partition.Definitions[0].ID
c.Assert(oldPID != newPID, IsTrue)

s.tk.MustExec("alter table employees truncate partition p3")
result = s.tk.MustQuery("select * from employees")
result.Check(testkit.Rows(`2 1995`))

s.tk.MustExec("insert into employees values (1, 1984)")
result = s.tk.MustQuery("select * from employees order by id")
result.Check(testkit.Rows(`1 1984`, `2 1995`))
s.tk.MustExec("insert into employees values (3, 2000)")
result = s.tk.MustQuery("select * from employees order by id")
result.Check(testkit.Rows(`1 1984`, `2 1995`, `3 2000`))

s.tk.MustExec(`create table non_partition (id int)`)
s.testErrorCode(c, "alter table non_partition truncate partition p0", tmysql.ErrPartitionMgmtOnNonpartitioned)
}

func (s *testDBSuite) TestAddPartitionTooManyPartitions(c *C) {
s.tk = testkit.NewTestKit(c, s.store)
s.tk.MustExec("use test")
Expand Down
3 changes: 3 additions & 0 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -83,6 +83,7 @@ var (
errWaitReorgTimeout = terror.ClassDDL.New(codeWaitReorgTimeout, "wait for reorganization timeout")
errInvalidStoreVer = terror.ClassDDL.New(codeInvalidStoreVer, "invalid storage current version")

errUnknownPartition = terror.ClassDDL.New(codeUnknownPartition, mysql.MySQLErrName[mysql.ErrUnknownPartition])
// We don't support dropping column with index covered now.
errCantDropColWithIndex = terror.ClassDDL.New(codeCantDropColWithIndex, "can't drop column with index")
errUnsupportedAddColumn = terror.ClassDDL.New(codeUnsupportedAddColumn, "unsupported add column")
Expand Down Expand Up @@ -636,6 +637,7 @@ const (
codeWrongExprInPartitionFunc = terror.ErrCode(mysql.ErrWrongExprInPartitionFunc)
codeWarnDataTruncated = terror.ErrCode(mysql.WarnDataTruncated)
codeCoalesceOnlyOnHashPartition = terror.ErrCode(mysql.ErrCoalesceOnlyOnHashPartition)
codeUnknownPartition = terror.ErrCode(mysql.ErrUnknownPartition)
)

func init() {
Expand Down Expand Up @@ -685,6 +687,7 @@ func init() {
codeWrongExprInPartitionFunc: mysql.ErrWrongExprInPartitionFunc,
codeWarnDataTruncated: mysql.WarnDataTruncated,
codeCoalesceOnlyOnHashPartition: mysql.ErrCoalesceOnlyOnHashPartition,
codeUnknownPartition: mysql.ErrUnknownPartition,
}
terror.ErrClassToMySQLCodes[terror.ClassDDL] = ddlMySQLErrCodes
}
39 changes: 39 additions & 0 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1154,6 +1154,8 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A
err = d.DropIndex(ctx, ident, model.NewCIStr(spec.Name))
case ast.AlterTableDropPartition:
err = d.DropTablePartition(ctx, ident, spec)
case ast.AlterTableTruncatePartition:
err = d.TruncateTablePartition(ctx, ident, spec)
case ast.AlterTableAddConstraint:
constr := spec.Constraint
switch spec.Constraint.Tp {
Expand Down Expand Up @@ -1470,6 +1472,43 @@ func (d *ddl) CoalescePartitions(ctx sessionctx.Context, ident ast.Ident, spec *
return errors.Trace(err)
}

func (d *ddl) TruncateTablePartition(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error {
is := d.infoHandle.Get()
schema, ok := is.SchemaByName(ident.Schema)
if !ok {
return errors.Trace(infoschema.ErrDatabaseNotExists.GenWithStackByArgs(schema))
}
t, err := is.TableByName(ident.Schema, ident.Name)
if err != nil {
return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ident.Schema, ident.Name))
}
meta := t.Meta()
if meta.GetPartitionInfo() == nil {
return errors.Trace(ErrPartitionMgmtOnNonpartitioned)
}

var pid int64
pid, err = findPartitionByName(meta, spec.Name)
if err != nil {
return errors.Trace(err)
}

job := &model.Job{
SchemaID: schema.ID,
TableID: meta.ID,
Type: model.ActionTruncateTablePartition,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{pid},
}

err = d.doDDLJob(ctx, job)
if err != nil {
return errors.Trace(err)
}
err = d.callHookOnChanged(err)
return errors.Trace(err)
}

func (d *ddl) DropTablePartition(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error {
is := d.infoHandle.Get()
schema, ok := is.SchemaByName(ident.Schema)
Expand Down
4 changes: 3 additions & 1 deletion ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -285,7 +285,7 @@ func (w *worker) finishDDLJob(t *meta.Meta, job *model.Job) (err error) {

// After rolling back an AddIndex operation, we need to use delete-range to delete the half-done index data.
err = w.deleteRange(job)
case model.ActionDropSchema, model.ActionDropTable, model.ActionTruncateTable, model.ActionDropIndex, model.ActionDropTablePartition:
case model.ActionDropSchema, model.ActionDropTable, model.ActionTruncateTable, model.ActionDropIndex, model.ActionDropTablePartition, model.ActionTruncateTablePartition:
err = w.deleteRange(job)
}
if err != nil {
Expand Down Expand Up @@ -472,6 +472,8 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64,
ver, err = onDropTable(t, job)
case model.ActionDropTablePartition:
ver, err = onDropTablePartition(t, job)
case model.ActionTruncateTablePartition:
ver, err = onTruncateTablePartition(t, job)
case model.ActionAddColumn:
ver, err = onAddColumn(d, t, job)
case model.ActionDropColumn:
Expand Down
2 changes: 1 addition & 1 deletion ddl/delete_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -243,7 +243,7 @@ func insertJobIntoDeleteRangeTable(ctx sessionctx.Context, job *model.Job) error
startKey = tablecodec.EncodeTablePrefix(tableID)
endKey := tablecodec.EncodeTablePrefix(tableID + 1)
return doInsert(s, job.ID, tableID, startKey, endKey, now)
case model.ActionDropTablePartition:
case model.ActionDropTablePartition, model.ActionTruncateTablePartition:
var physicalTableID int64
if err := job.DecodeArgs(&physicalTableID); err != nil {
return errors.Trace(err)
Expand Down
58 changes: 58 additions & 0 deletions ddl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -318,6 +318,18 @@ func checkDropTablePartition(meta *model.TableInfo, partName string) error {
return errors.Trace(ErrDropPartitionNonExistent.GenWithStackByArgs(partName))
}

func findPartitionByName(meta *model.TableInfo, parName string) (int64, error) {
// TODO: MySQL behavior for hash partition is weird, "create table .. partition by hash partition 4",
// it use p0, p1, p2, p3 as partition names automatically.
parName = strings.ToLower(parName)
for _, def := range meta.Partition.Definitions {
if strings.EqualFold(def.Name.L, parName) {
return def.ID, nil
}
}
return -1, errors.Trace(errUnknownPartition.GenWithStackByArgs(parName, meta.Name.O))
}

// removePartitionInfo each ddl job deletes a partition.
func removePartitionInfo(tblInfo *model.TableInfo, partName string) int64 {
oldDefs := tblInfo.Partition.Definitions
Expand Down Expand Up @@ -365,6 +377,52 @@ func onDropTablePartition(t *meta.Meta, job *model.Job) (ver int64, _ error) {
return ver, nil
}

// onDropTablePartition truncates old partition meta.
func onTruncateTablePartition(t *meta.Meta, job *model.Job) (int64, error) {
var ver int64
var oldID int64
if err := job.DecodeArgs(&oldID); err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}
tblInfo, err := getTableInfo(t, job, job.SchemaID)
if err != nil {
return ver, errors.Trace(err)
}
pi := tblInfo.GetPartitionInfo()
if pi == nil {
return ver, errors.Trace(ErrPartitionMgmtOnNonpartitioned)
}

var find bool
for i := 0; i < len(pi.Definitions); i++ {
def := &pi.Definitions[i]
if def.ID == oldID {
pid, err1 := t.GenGlobalID()
if err != nil {
return ver, errors.Trace(err1)
}
def.ID = pid
find = true
break
}
}
if !find {
return ver, errUnknownPartition.GenWithStackByArgs("drop?", tblInfo.Name.O)
}

ver, err = updateVersionAndTableInfo(t, job, tblInfo, true)
if err != nil {
return ver, errors.Trace(err)
}

// Finish this job.
job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo)
// A background job will be created to delete old partition data.
job.Args = []interface{}{oldID}
return ver, nil
}

func checkAddPartitionTooManyPartitions(piDefs uint64) error {
if piDefs > uint64(PartitionCountLimit) {
return ErrTooManyPartitions
Expand Down
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ require (
github.com/pingcap/errors v0.11.0
github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e
github.com/pingcap/kvproto v0.0.0-20181105061835-1b5d69cd1d26
github.com/pingcap/parser v0.0.0-20181210061630-27e9d3e251d4
github.com/pingcap/parser v0.0.0-20181211024540-4e6d047fcaae
github.com/pingcap/pd v2.1.0-rc.4+incompatible
github.com/pingcap/tidb-tools v0.0.0-20181112132202-4860a0d5de03
github.com/pingcap/tipb v0.0.0-20181012112600-11e33c750323
Expand Down
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -107,8 +107,8 @@ github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e h1:P73/4dPCL96rG
github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw=
github.com/pingcap/kvproto v0.0.0-20181105061835-1b5d69cd1d26 h1:JK4VLNYbSn36QSbCnqALi2ySXdH0DfcMssT/zmLf4Ls=
github.com/pingcap/kvproto v0.0.0-20181105061835-1b5d69cd1d26/go.mod h1:0gwbe1F2iBIjuQ9AH0DbQhL+Dpr5GofU8fgYyXk+ykk=
github.com/pingcap/parser v0.0.0-20181210061630-27e9d3e251d4 h1:2rCHDk4h8VZw0fiC2CFJffOlXU3iMuz1kOt5wTMCemY=
github.com/pingcap/parser v0.0.0-20181210061630-27e9d3e251d4/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA=
github.com/pingcap/parser v0.0.0-20181211024540-4e6d047fcaae h1:RD98+89F/yakFLnztEL4Pi9f+RkOAm2vgRJcB1p6tTw=
github.com/pingcap/parser v0.0.0-20181211024540-4e6d047fcaae/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA=
github.com/pingcap/pd v2.1.0-rc.4+incompatible h1:/buwGk04aHO5odk/+O8ZOXGs4qkUjYTJ2UpCJXna8NE=
github.com/pingcap/pd v2.1.0-rc.4+incompatible/go.mod h1:nD3+EoYes4+aNNODO99ES59V83MZSI+dFbhyr667a0E=
github.com/pingcap/tidb-tools v0.0.0-20181112132202-4860a0d5de03 h1:xVuo5U+l6XAWHsb+xhkZ8zz3jerIwDfCHAO6kR2Kaog=
Expand Down

0 comments on commit 8c9e4be

Please sign in to comment.