diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 4f64f3164496e..786401ff773b0 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -1210,3 +1210,79 @@ func (s *testIntegrationSuite) TestAlterColumn(c *C) { _, err = s.tk.Exec("alter table mc modify column a bigint auto_increment") // Adds auto_increment should throw error c.Assert(err, NotNil) } + +func (s *testIntegrationSuite) assertWarningExec(c *C, sql string, expectedWarn *terror.Error) { + _, err := s.tk.Exec(sql) + c.Assert(err, IsNil) + st := s.tk.Se.GetSessionVars().StmtCtx + c.Assert(st.WarningCount(), Equals, uint16(1)) + c.Assert(expectedWarn.Equal(st.GetWarnings()[0].Err), IsTrue, Commentf("error:%v", err)) +} + +func (s *testIntegrationSuite) assertAlterWarnExec(c *C, sql string) { + s.assertWarningExec(c, sql, ddl.ErrAlterOperationNotSupported) +} + +func (s *testIntegrationSuite) TestAlterAlgorithm(c *C) { + s.tk = testkit.NewTestKit(c, s.store) + s.tk.MustExec("use test") + s.tk.MustExec("drop table if exists t") + s.tk.MustExec("drop table if exists t1") + defer s.tk.MustExec("drop table if exists t") + + s.tk.MustExec(`create table t( + a int, + b varchar(100), + c int, + INDEX idx_c(c)) PARTITION BY RANGE ( a ) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11), + PARTITION p2 VALUES LESS THAN (16), + PARTITION p3 VALUES LESS THAN (21) + )`) + s.assertAlterWarnExec(c, "alter table t modify column a bigint, ALGORITHM=INPLACE;") + s.tk.MustExec("alter table t modify column a bigint, ALGORITHM=INPLACE, ALGORITHM=INSTANT;") + s.tk.MustExec("alter table t modify column a bigint, ALGORITHM=DEFAULT;") + + // Test add/drop index + s.assertAlterWarnExec(c, "alter table t add index idx_b(b), ALGORITHM=INSTANT") + s.assertAlterWarnExec(c, "alter table t add index idx_b1(b), ALGORITHM=COPY") + s.tk.MustExec("alter table t add index idx_b2(b), ALGORITHM=INPLACE") + s.assertAlterWarnExec(c, "alter table t drop index idx_b, ALGORITHM=INPLACE") + s.assertAlterWarnExec(c, "alter table t drop index idx_b1, ALGORITHM=COPY") + s.tk.MustExec("alter table t drop index idx_b2, ALGORITHM=INSTANT") + + // Test rename + s.assertAlterWarnExec(c, "alter table t rename to t1, ALGORITHM=COPY") + s.assertAlterWarnExec(c, "alter table t1 rename to t, ALGORITHM=INPLACE") + s.tk.MustExec("alter table t rename to t1, ALGORITHM=INSTANT") + s.tk.MustExec("alter table t1 rename to t, ALGORITHM=DEFAULT") + + // Test rename index + s.assertAlterWarnExec(c, "alter table t rename index idx_c to idx_c1, ALGORITHM=COPY") + s.assertAlterWarnExec(c, "alter table t rename index idx_c1 to idx_c, ALGORITHM=INPLACE") + s.tk.MustExec("alter table t rename index idx_c to idx_c1, ALGORITHM=INSTANT") + s.tk.MustExec("alter table t rename index idx_c1 to idx_c, ALGORITHM=DEFAULT") + + // partition. + s.assertAlterWarnExec(c, "alter table t truncate partition p1, ALGORITHM=COPY") + s.assertAlterWarnExec(c, "alter table t truncate partition p2, ALGORITHM=INPLACE") + s.tk.MustExec("alter table t truncate partition p3, ALGORITHM=INSTANT") + + s.assertAlterWarnExec(c, "alter table t add partition (partition p4 values less than (2002)), ALGORITHM=COPY") + s.assertAlterWarnExec(c, "alter table t add partition (partition p5 values less than (3002)), ALGORITHM=INPLACE") + s.tk.MustExec("alter table t add partition (partition p6 values less than (4002)), ALGORITHM=INSTANT") + + s.assertAlterWarnExec(c, "alter table t drop partition p4, ALGORITHM=COPY") + s.assertAlterWarnExec(c, "alter table t drop partition p5, ALGORITHM=INPLACE") + s.tk.MustExec("alter table t drop partition p6, ALGORITHM=INSTANT") + + // Table options + s.assertAlterWarnExec(c, "alter table t comment = 'test', ALGORITHM=COPY") + s.assertAlterWarnExec(c, "alter table t comment = 'test', ALGORITHM=INPLACE") + s.tk.MustExec("alter table t comment = 'test', ALGORITHM=INSTANT") + + s.assertAlterWarnExec(c, "alter table t default charset = utf8mb4, ALGORITHM=COPY") + s.assertAlterWarnExec(c, "alter table t default charset = utf8mb4, ALGORITHM=INPLACE") + s.tk.MustExec("alter table t default charset = utf8mb4, ALGORITHM=INSTANT") +} diff --git a/ddl/ddl.go b/ddl/ddl.go index 2980fe8c93483..d63aa841d7a1b 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -203,6 +203,8 @@ var ( ErrCoalesceOnlyOnHashPartition = terror.ClassDDL.New(codeCoalesceOnlyOnHashPartition, mysql.MySQLErrName[mysql.ErrCoalesceOnlyOnHashPartition]) // ErrViewWrongList returns create view must include all columns in the select clause ErrViewWrongList = terror.ClassDDL.New(codeViewWrongList, mysql.MySQLErrName[mysql.ErrViewWrongList]) + // ErrAlterOperationNotSupported returns when alter operations is not supported. + ErrAlterOperationNotSupported = terror.ClassDDL.New(codeNotSupportedAlterOperation, mysql.MySQLErrName[mysql.ErrAlterOperationNotSupportedReason]) // ErrWrongObject returns for wrong object. ErrWrongObject = terror.ClassDDL.New(codeErrWrongObject, mysql.MySQLErrName[mysql.ErrWrongObject]) ) @@ -685,6 +687,7 @@ const ( codeWarnDataTruncated = terror.ErrCode(mysql.WarnDataTruncated) codeCoalesceOnlyOnHashPartition = terror.ErrCode(mysql.ErrCoalesceOnlyOnHashPartition) codeUnknownPartition = terror.ErrCode(mysql.ErrUnknownPartition) + codeNotSupportedAlterOperation = terror.ErrCode(mysql.ErrAlterOperationNotSupportedReason) ) func init() { @@ -736,6 +739,7 @@ func init() { codeWarnDataTruncated: mysql.WarnDataTruncated, codeCoalesceOnlyOnHashPartition: mysql.ErrCoalesceOnlyOnHashPartition, codeUnknownPartition: mysql.ErrUnknownPartition, + codeNotSupportedAlterOperation: mysql.ErrAlterOperationNotSupportedReason, codeErrWrongObject: mysql.ErrWrongObject, } terror.ErrClassToMySQLCodes[terror.ClassDDL] = ddlMySQLErrCodes diff --git a/ddl/ddl_algorithm.go b/ddl/ddl_algorithm.go new file mode 100644 index 0000000000000..cb8fa321c3693 --- /dev/null +++ b/ddl/ddl_algorithm.go @@ -0,0 +1,72 @@ +// Copyright 2018 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package ddl + +import ( + "fmt" + + "github.com/pingcap/parser/ast" +) + +// AlterAlgorithm is used to store supported alter algorithm. +// For now, TiDB only support AlterAlgorithmInplace and AlterAlgorithmInstant. +// The most alter operations are using instant algorithm, and only the add index is using inplace(not really inplace, +// because we never block the DML but costs some time to backfill the index data) +// See https://dev.mysql.com/doc/refman/8.0/en/alter-table.html#alter-table-performance. +type AlterAlgorithm struct { + supported []ast.AlterAlgorithm + // If the alter algorithm is not given, the defAlgorithm will be used. + defAlgorithm ast.AlterAlgorithm +} + +var ( + instantAlgorithm = &AlterAlgorithm{ + supported: []ast.AlterAlgorithm{ast.AlterAlgorithmInstant}, + defAlgorithm: ast.AlterAlgorithmInstant, + } + + inplaceAlgorithm = &AlterAlgorithm{ + supported: []ast.AlterAlgorithm{ast.AlterAlgorithmInplace}, + defAlgorithm: ast.AlterAlgorithmInplace, + } + + defaultAlgorithm = ast.AlterAlgorithmInstant +) + +func getProperAlgorithm(specify ast.AlterAlgorithm, algorithm *AlterAlgorithm) (ast.AlterAlgorithm, error) { + if specify == ast.AlterAlgorithmDefault { + return algorithm.defAlgorithm, nil + } + + for _, a := range algorithm.supported { + if specify == a { + return specify, nil + } + } + + return algorithm.defAlgorithm, ErrAlterOperationNotSupported.GenWithStackByArgs(fmt.Sprintf("ALGORITHM=%s", specify), fmt.Sprintf("Cannot alter table by %s", specify), fmt.Sprintf("ALGORITHM=%s", algorithm.defAlgorithm)) +} + +// ResolveAlterAlgorithm resolves the algorithm of the alterSpec. +// If specify algorithm is not supported by the alter action, errAlterOperationNotSupported will be returned. +// If specify is the ast.AlterAlgorithmDefault, then the default algorithm of the alter action will be returned. +func ResolveAlterAlgorithm(alterSpec *ast.AlterTableSpec, specify ast.AlterAlgorithm) (ast.AlterAlgorithm, error) { + switch alterSpec.Tp { + // For now, TiDB only support inplace algorithm and instant algorithm. + case ast.AlterTableAddConstraint: + return getProperAlgorithm(specify, inplaceAlgorithm) + default: + return getProperAlgorithm(specify, instantAlgorithm) + } +} diff --git a/ddl/ddl_algorithm_test.go b/ddl/ddl_algorithm_test.go new file mode 100644 index 0000000000000..1d1a49e642daf --- /dev/null +++ b/ddl/ddl_algorithm_test.go @@ -0,0 +1,107 @@ +// Copyright 2018 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package ddl_test + +import ( + . "github.com/pingcap/check" + "github.com/pingcap/parser/ast" + "github.com/pingcap/tidb/ddl" +) + +var _ = Suite(&testDDLAlgorithmSuite{}) + +var ( + allAlgorithm = []ast.AlterAlgorithm{ast.AlterAlgorithmCopy, + ast.AlterAlgorithmInplace, ast.AlterAlgorithmInstant} +) + +type testDDLAlgorithmSuite struct{} + +type testCase struct { + alterSpec ast.AlterTableSpec + supportedAlgorithm []ast.AlterAlgorithm + defAlgorithm ast.AlterAlgorithm +} + +func (s *testDDLAlgorithmSuite) TestFindAlterAlgorithm(c *C) { + instantAlgorithm := []ast.AlterAlgorithm{ast.AlterAlgorithmInstant} + inplaceAlgorithm := []ast.AlterAlgorithm{ast.AlterAlgorithmInplace} + + testCases := []testCase{ + {ast.AlterTableSpec{Tp: ast.AlterTableAddConstraint}, inplaceAlgorithm, ast.AlterAlgorithmInplace}, + {ast.AlterTableSpec{Tp: ast.AlterTableAddColumns}, instantAlgorithm, ast.AlterAlgorithmInstant}, + {ast.AlterTableSpec{Tp: ast.AlterTableDropColumn}, instantAlgorithm, ast.AlterAlgorithmInstant}, + {ast.AlterTableSpec{Tp: ast.AlterTableDropPrimaryKey}, instantAlgorithm, ast.AlterAlgorithmInstant}, + {ast.AlterTableSpec{Tp: ast.AlterTableDropIndex}, instantAlgorithm, ast.AlterAlgorithmInstant}, + {ast.AlterTableSpec{Tp: ast.AlterTableDropForeignKey}, instantAlgorithm, ast.AlterAlgorithmInstant}, + {ast.AlterTableSpec{Tp: ast.AlterTableRenameTable}, instantAlgorithm, ast.AlterAlgorithmInstant}, + {ast.AlterTableSpec{Tp: ast.AlterTableRenameIndex}, instantAlgorithm, ast.AlterAlgorithmInstant}, + + // Alter table options. + {ast.AlterTableSpec{Tp: ast.AlterTableOption, Options: []*ast.TableOption{{Tp: ast.TableOptionShardRowID}}}, instantAlgorithm, ast.AlterAlgorithmInstant}, + {ast.AlterTableSpec{Tp: ast.AlterTableOption, Options: []*ast.TableOption{{Tp: ast.TableOptionAutoIncrement}}}, instantAlgorithm, ast.AlterAlgorithmInstant}, + {ast.AlterTableSpec{Tp: ast.AlterTableOption, Options: []*ast.TableOption{{Tp: ast.TableOptionComment}}}, instantAlgorithm, ast.AlterAlgorithmInstant}, + {ast.AlterTableSpec{Tp: ast.AlterTableOption, Options: []*ast.TableOption{{Tp: ast.TableOptionCharset}}}, instantAlgorithm, ast.AlterAlgorithmInstant}, + {ast.AlterTableSpec{Tp: ast.AlterTableOption, Options: []*ast.TableOption{{Tp: ast.TableOptionCollate}}}, instantAlgorithm, ast.AlterAlgorithmInstant}, + + // TODO: after we support migrate the data of partitions, change below cases. + {ast.AlterTableSpec{Tp: ast.AlterTableCoalescePartitions}, instantAlgorithm, ast.AlterAlgorithmInstant}, + {ast.AlterTableSpec{Tp: ast.AlterTableAddPartitions}, instantAlgorithm, ast.AlterAlgorithmInstant}, + {ast.AlterTableSpec{Tp: ast.AlterTableDropPartition}, instantAlgorithm, ast.AlterAlgorithmInstant}, + {ast.AlterTableSpec{Tp: ast.AlterTableTruncatePartition}, instantAlgorithm, ast.AlterAlgorithmInstant}, + + // TODO: after we support lock a table, change the below case. + {ast.AlterTableSpec{Tp: ast.AlterTableLock}, instantAlgorithm, ast.AlterAlgorithmInstant}, + // TODO: after we support changing the column type, below cases need to change. + {ast.AlterTableSpec{Tp: ast.AlterTableModifyColumn}, instantAlgorithm, ast.AlterAlgorithmInstant}, + {ast.AlterTableSpec{Tp: ast.AlterTableChangeColumn}, instantAlgorithm, ast.AlterAlgorithmInstant}, + {ast.AlterTableSpec{Tp: ast.AlterTableAlterColumn}, instantAlgorithm, ast.AlterAlgorithmInstant}, + } + + for _, tc := range testCases { + runAlterAlgorithmTestCases(c, &tc) + } +} + +func runAlterAlgorithmTestCases(c *C, tc *testCase) { + algorithm, err := ddl.ResolveAlterAlgorithm(&tc.alterSpec, ast.AlterAlgorithmDefault) + c.Assert(err, IsNil) + c.Assert(algorithm, Equals, tc.defAlgorithm) + + unsupported := make([]ast.AlterAlgorithm, 0, len(allAlgorithm)) +Loop: + for _, alm := range allAlgorithm { + for _, almSupport := range tc.supportedAlgorithm { + if alm == almSupport { + continue Loop + } + } + + unsupported = append(unsupported, alm) + } + + // Test supported. + for _, alm := range tc.supportedAlgorithm { + algorithm, err = ddl.ResolveAlterAlgorithm(&tc.alterSpec, alm) + c.Assert(err, IsNil) + c.Assert(algorithm, Equals, alm) + } + + // Test unsupported. + for _, alm := range unsupported { + algorithm, err = ddl.ResolveAlterAlgorithm(&tc.alterSpec, alm) + c.Assert(err, NotNil, Commentf("Tp:%v, alm:%s", tc.alterSpec.Tp, alm)) + c.Assert(ddl.ErrAlterOperationNotSupported.Equal(err), IsTrue) + } +} diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 555296c5e4ae6..7e31ba95b8966 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1424,10 +1424,16 @@ func getCharsetAndCollateInTableOption(startIdx int, options []*ast.TableOption) return } -func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.AlterTableSpec) (err error) { - // Only handle valid specs. +// resolveAlterTableSpec resolves alter table algorithm and removes ignore table spec in specs. +// returns valied specs, and the occured error. +func resolveAlterTableSpec(ctx sessionctx.Context, specs []*ast.AlterTableSpec) ([]*ast.AlterTableSpec, error) { validSpecs := make([]*ast.AlterTableSpec, 0, len(specs)) + algorithm := ast.AlterAlgorithmDefault for _, spec := range specs { + if spec.Tp == ast.AlterTableAlgorithm { + // Find the last AlterTableAlgorithm. + algorithm = spec.Algorithm + } if isIgnorableSpec(spec.Tp) { continue } @@ -1437,7 +1443,29 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A if len(validSpecs) != 1 { // TODO: Hanlde len(validSpecs) == 0. // Now we only allow one schema changing at the same time. - return errRunMultiSchemaChanges + return nil, errRunMultiSchemaChanges + } + + // Verify whether the algorithm is supported. + for _, spec := range validSpecs { + algorithm, err := ResolveAlterAlgorithm(spec, algorithm) + if err != nil { + // For the compatibility, we return warning instead of error. + ctx.GetSessionVars().StmtCtx.AppendError(err) + err = nil + } + + spec.Algorithm = algorithm + } + + // Only handle valid specs. + return validSpecs, nil +} + +func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.AlterTableSpec) (err error) { + validSpecs, err := resolveAlterTableSpec(ctx, specs) + if err != nil { + return errors.Trace(err) } is := d.infoHandle.Get() diff --git a/go.mod b/go.mod index 68efd2eee1db2..a488b92084aa0 100644 --- a/go.mod +++ b/go.mod @@ -49,7 +49,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e github.com/pingcap/kvproto v0.0.0-20190215154024-7f2fc73ef562 github.com/pingcap/log v0.0.0-20190214045112-b37da76f67a7 - github.com/pingcap/parser v0.0.0-20190218033509-9545f168ae97 + github.com/pingcap/parser v0.0.0-20190219084020-362712b5ab93 github.com/pingcap/pd v2.1.0-rc.4+incompatible github.com/pingcap/tidb-tools v2.1.3-0.20190116051332-34c808eef588+incompatible github.com/pingcap/tipb v0.0.0-20190107072121-abbec73437b7 diff --git a/go.sum b/go.sum index 403c6e5a729c6..510801c87d261 100644 --- a/go.sum +++ b/go.sum @@ -113,8 +113,8 @@ github.com/pingcap/kvproto v0.0.0-20190215154024-7f2fc73ef562 h1:32oF1/8lVnBR2JV github.com/pingcap/kvproto v0.0.0-20190215154024-7f2fc73ef562/go.mod h1:QMdbTAXCHzzygQzqcG9uVUgU2fKeSN1GmfMiykdSzzY= github.com/pingcap/log v0.0.0-20190214045112-b37da76f67a7 h1:kOHAMalwF69bJrtWrOdVaCSvZjLucrJhP4NQKIu6uM4= github.com/pingcap/log v0.0.0-20190214045112-b37da76f67a7/go.mod h1:xsfkWVaFVV5B8e1K9seWfyJWFrIhbtUTAD8NV1Pq3+w= -github.com/pingcap/parser v0.0.0-20190218033509-9545f168ae97 h1:GIhPQAwFwnf6cSdVYXdSNkx171Nl9ZmIVYrOtN3I2lw= -github.com/pingcap/parser v0.0.0-20190218033509-9545f168ae97/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= +github.com/pingcap/parser v0.0.0-20190219084020-362712b5ab93 h1:jRnleVTpcisLEHNRYHBu4ilwEPyUkslJJfhYNEQlz2g= +github.com/pingcap/parser v0.0.0-20190219084020-362712b5ab93/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 v2.1.3-0.20190116051332-34c808eef588+incompatible h1:e9Gi/LP9181HT3gBfSOeSBA+5JfemuE4aEAhqNgoE4k=