diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 4f6e7b2c7fc98..8880fc9e4d707 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -1613,17 +1613,17 @@ func (s *testIntegrationSuite3) TestAlterAlgorithm(c *C) { s.tk.MustExec("alter table t rename index idx_c to idx_c1, ALGORITHM=DEFAULT") // partition. - s.assertAlterWarnExec(c, "alter table t truncate partition p1, ALGORITHM=COPY") - s.assertAlterErrorExec(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 ALGORITHM=COPY, truncate partition p1") + s.assertAlterErrorExec(c, "alter table t ALGORITHM=INPLACE, truncate partition p2") + s.tk.MustExec("alter table t ALGORITHM=INSTANT, truncate partition p3") s.assertAlterWarnExec(c, "alter table t add partition (partition p4 values less than (2002)), ALGORITHM=COPY") s.assertAlterErrorExec(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.assertAlterErrorExec(c, "alter table t drop partition p5, ALGORITHM=INPLACE") - s.tk.MustExec("alter table t drop partition p6, ALGORITHM=INSTANT") + s.assertAlterWarnExec(c, "alter table t ALGORITHM=COPY, drop partition p4") + s.assertAlterErrorExec(c, "alter table t ALGORITHM=INPLACE, drop partition p5") + s.tk.MustExec("alter table t ALGORITHM=INSTANT, drop partition p6") // Table options s.assertAlterWarnExec(c, "alter table t comment = 'test', ALGORITHM=COPY") diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 4df2376118885..e2810a620cfcb 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -23,6 +23,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" + "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" tmysql "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" @@ -324,14 +325,18 @@ create table log_message_1 ( cases := []testCase{ { "create table t (id int) partition by range columns (id);", - ddl.ErrPartitionsMustBeDefined, + ast.ErrPartitionsMustBeDefined, }, { "create table t (id int) partition by range columns (id) (partition p0 values less than (1, 2));", - ddl.ErrPartitionColumnList, + ast.ErrPartitionColumnList, }, { "create table t (a int) partition by range columns (b) (partition p0 values less than (1, 2));", + ast.ErrPartitionColumnList, + }, + { + "create table t (a int) partition by range columns (b) (partition p0 values less than (1));", ddl.ErrFieldNotFoundPart, }, { @@ -371,7 +376,10 @@ create table log_message_1 ( } for i, t := range cases { _, err := tk.Exec(t.sql) - c.Assert(t.err.Equal(err), IsTrue, Commentf("case %d fail, sql = %s", i, t.sql)) + c.Assert(t.err.Equal(err), IsTrue, Commentf( + "case %d fail, sql = `%s`\nexpected error = `%v`\n actual error = `%v`", + i, t.sql, t.err, err, + )) } tk.MustExec("create table t1 (a int, b char(3)) partition by range columns (a, b) (" + @@ -496,6 +504,15 @@ func (s *testIntegrationSuite5) TestAlterTableAddPartition(c *C) { );` assertErrorCode(c, tk, sql7, tmysql.ErrSameNamePartition) + sql8 := "alter table table3 add partition (partition p6);" + assertErrorCode(c, tk, sql8, tmysql.ErrPartitionRequiresValues) + + sql9 := "alter table table3 add partition (partition p7 values in (2018));" + assertErrorCode(c, tk, sql9, tmysql.ErrPartitionWrongValues) + + sql10 := "alter table table3 add partition partitions 4;" + assertErrorCode(c, tk, sql10, tmysql.ErrPartitionsMustBeDefined) + tk.MustExec("alter table table3 add partition (partition p3 values less than (2001 + 10))") // less than value can be negative or expression. @@ -813,7 +830,7 @@ func (s *testIntegrationSuite6) TestTruncatePartitionAndDropTable(c *C) { tk.MustExec("drop table if exists t5;") tk.MustExec("set @@session.tidb_enable_table_partition=1;") tk.MustExec(`create table t5( - id int, name varchar(50), + id int, name varchar(50), purchased date ) partition by range( year(purchased) ) ( @@ -1472,3 +1489,34 @@ func (s *testIntegrationSuite4) TestPartitionErrorCode(c *C) { _, err = tk.Exec("alter table t_part coalesce partition 4;") c.Assert(ddl.ErrCoalesceOnlyOnHashPartition.Equal(err), IsTrue) } + +func (s *testIntegrationSuite3) TestUnsupportedPartitionManagementDDLs(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists test_1465;") + tk.MustExec(` + create table test_1465 (a int) + partition by range(a) ( + partition p1 values less than (10), + partition p2 values less than (20), + partition p3 values less than (30) + ); + `) + + _, err := tk.Exec("alter table test_1465 truncate partition p1, p2") + c.Assert(err, ErrorMatches, ".*can't run multi schema change") + _, err = tk.Exec("alter table test_1465 drop partition p1, p2") + c.Assert(err, ErrorMatches, ".*can't run multi schema change") + + _, err = tk.Exec("alter table test_1465 partition by hash(a)") + c.Assert(err, ErrorMatches, ".*alter table partition is unsupported") +} + +func (s *testIntegrationSuite8) TestTruncateTableWithPartition(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists test_4414;") + tk.MustExec("create table test_4414(a int, b int) partition by hash(a) partitions 10;") + tk.MustExec("truncate table test_4414;") + tk.MustQuery("select * from test_4414 partition (p0)").Check(testkit.Rows()) +} diff --git a/ddl/ddl.go b/ddl/ddl.go index 8cc1b5d840769..a5bb7b6637088 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -194,8 +194,6 @@ var ( // ErrNotAllowedTypeInPartition returns not allowed type error when creating table partiton with unsupport expression type. ErrNotAllowedTypeInPartition = terror.ClassDDL.New(codeErrFieldTypeNotAllowedAsPartitionField, mysql.MySQLErrName[mysql.ErrFieldTypeNotAllowedAsPartitionField]) - // ErrPartitionsMustBeDefined returns each partition must be defined. - ErrPartitionsMustBeDefined = terror.ClassDDL.New(codePartitionsMustBeDefined, "For RANGE partitions each partition must be defined") // ErrPartitionMgmtOnNonpartitioned returns it's not a partition table. ErrPartitionMgmtOnNonpartitioned = terror.ClassDDL.New(codePartitionMgmtOnNonpartitioned, "Partition management on a not partitioned table is not possible") // ErrDropPartitionNonExistent returns error in list of partition. @@ -206,14 +204,10 @@ var ( ErrRangeNotIncreasing = terror.ClassDDL.New(codeRangeNotIncreasing, "VALUES LESS THAN value must be strictly increasing for each partition") // ErrPartitionMaxvalue returns maxvalue can only be used in last partition definition. ErrPartitionMaxvalue = terror.ClassDDL.New(codePartitionMaxvalue, "MAXVALUE can only be used in last partition definition") - // ErrTooManyValues returns cannot have more than one value for this type of partitioning. - ErrTooManyValues = terror.ClassDDL.New(codeErrTooManyValues, mysql.MySQLErrName[mysql.ErrTooManyValues]) //ErrDropLastPartition returns cannot remove all partitions, use drop table instead. ErrDropLastPartition = terror.ClassDDL.New(codeDropLastPartition, mysql.MySQLErrName[mysql.ErrDropLastPartition]) //ErrTooManyPartitions returns too many partitions were defined. ErrTooManyPartitions = terror.ClassDDL.New(codeTooManyPartitions, mysql.MySQLErrName[mysql.ErrTooManyPartitions]) - //ErrNoParts returns no partition were defined. - ErrNoParts = terror.ClassDDL.New(codeNoParts, mysql.MySQLErrName[mysql.ErrNoParts]) //ErrPartitionFunctionIsNotAllowed returns this partition function is not allowed. ErrPartitionFunctionIsNotAllowed = terror.ClassDDL.New(codePartitionFunctionIsNotAllowed, mysql.MySQLErrName[mysql.ErrPartitionFunctionIsNotAllowed]) // ErrPartitionFuncNotAllowed returns partition function returns the wrong type. @@ -235,8 +229,6 @@ var ( ErrTableCantHandleFt = terror.ClassDDL.New(codeErrTableCantHandleFt, mysql.MySQLErrName[mysql.ErrTableCantHandleFt]) // ErrFieldNotFoundPart returns an error when 'partition by columns' are not found in table columns. ErrFieldNotFoundPart = terror.ClassDDL.New(codeFieldNotFoundPart, mysql.MySQLErrName[mysql.ErrFieldNotFoundPart]) - // ErrPartitionColumnList returns "Inconsistency in usage of column lists for partitioning". - ErrPartitionColumnList = terror.ClassDDL.New(codePartitionColumnList, mysql.MySQLErrName[mysql.ErrPartitionColumnList]) ) // DDL is responsible for updating schema in data store and maintaining in-memory InfoSchema cache. @@ -750,6 +742,14 @@ const ( codeFieldNotFoundPart = terror.ErrCode(mysql.ErrFieldNotFoundPart) codePartitionColumnList = terror.ErrCode(mysql.ErrPartitionColumnList) codeOnlyOnRangeListPartition = terror.ErrCode(mysql.ErrOnlyOnRangeListPartition) + codePartitionRequiresValues = terror.ErrCode(mysql.ErrPartitionRequiresValues) + codePartitionWrongNoPart = terror.ErrCode(mysql.ErrPartitionWrongNoPart) + codePartitionWrongNoSubpart = terror.ErrCode(mysql.ErrPartitionWrongNoSubpart) + codePartitionWrongValues = terror.ErrCode(mysql.ErrPartitionWrongValues) + codeRowSinglePartitionField = terror.ErrCode(mysql.ErrRowSinglePartitionField) + codeSubpartition = terror.ErrCode(mysql.ErrSubpartition) + codeSystemVersioningWrongPartitions = terror.ErrCode(mysql.ErrSystemVersioningWrongPartitions) + codeWrongPartitionTypeExpectedSystemTime = terror.ErrCode(mysql.ErrWrongPartitionTypeExpectedSystemTime) ) func init() { @@ -813,6 +813,14 @@ func init() { codeInvalidDefaultValue: mysql.ErrInvalidDefault, codeErrGeneratedColumnRefAutoInc: mysql.ErrGeneratedColumnRefAutoInc, codeOnlyOnRangeListPartition: mysql.ErrOnlyOnRangeListPartition, + codePartitionRequiresValues: mysql.ErrPartitionRequiresValues, + codePartitionWrongNoPart: mysql.ErrPartitionWrongNoPart, + codePartitionWrongNoSubpart: mysql.ErrPartitionWrongNoSubpart, + codePartitionWrongValues: mysql.ErrPartitionWrongValues, + codeRowSinglePartitionField: mysql.ErrRowSinglePartitionField, + codeSubpartition: mysql.ErrSubpartition, + codeSystemVersioningWrongPartitions: mysql.ErrSystemVersioningWrongPartitions, + codeWrongPartitionTypeExpectedSystemTime: mysql.ErrWrongPartitionTypeExpectedSystemTime, } terror.ErrClassToMySQLCodes[terror.ClassDDL] = ddlMySQLErrCodes } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 016cbc589fd53..e1597411c03b7 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1583,12 +1583,12 @@ func checkRangeColumnsPartitionValue(ctx sessionctx.Context, tbInfo *model.Table // Range columns partition key supports multiple data types with integer、datetime、string. defs := pi.Definitions if len(defs) < 1 { - return errors.Trace(ErrPartitionsMustBeDefined) + return ast.ErrPartitionsMustBeDefined.GenWithStackByArgs("RANGE") } curr := &defs[0] if len(curr.LessThan) != len(pi.Columns) { - return errors.Trace(ErrPartitionColumnList) + return errors.Trace(ast.ErrPartitionColumnList) } for i := 1; i < len(defs); i++ { prev, curr := curr, &defs[i] @@ -1605,7 +1605,7 @@ func checkRangeColumnsPartitionValue(ctx sessionctx.Context, tbInfo *model.Table func checkTwoRangeColumns(ctx sessionctx.Context, curr, prev *model.PartitionDefinition, pi *model.PartitionInfo, tbInfo *model.TableInfo) (bool, error) { if len(curr.LessThan) != len(pi.Columns) { - return false, errors.Trace(ErrPartitionColumnList) + return false, errors.Trace(ast.ErrPartitionColumnList) } for i := 0; i < len(pi.Columns); i++ { // Special handling for MAXVALUE. @@ -1803,8 +1803,7 @@ func resolveAlterTableSpec(ctx sessionctx.Context, specs []*ast.AlterTableSpec) validSpecs = append(validSpecs, spec) } - if len(validSpecs) != 1 { - // TODO: Hanlde len(validSpecs) == 0. + if len(validSpecs) > 1 { // Now we only allow one schema changing at the same time. return nil, errRunMultiSchemaChanges } @@ -1891,6 +1890,9 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A err = ErrUnsupportedModifyPrimaryKey.GenWithStackByArgs("drop") case ast.AlterTableRenameIndex: err = d.RenameIndex(ctx, ident, spec) + case ast.AlterTablePartition: + // Prevent silent succeed if user executes ALTER TABLE x PARTITION BY ... + err = errors.New("alter table partition is unsupported") case ast.AlterTableOption: for i, opt := range spec.Options { switch opt.Tp { @@ -2118,10 +2120,6 @@ func (d *ddl) AddTablePartitions(ctx sessionctx.Context, ident ast.Ident, spec * if pi == nil { return errors.Trace(ErrPartitionMgmtOnNonpartitioned) } - // We don't support add hash type partition now. - if meta.Partition.Type == model.PartitionTypeHash { - return errors.Trace(ErrUnsupportedAddPartition) - } partInfo, err := buildPartitionInfo(meta, d, spec) if err != nil { @@ -2177,20 +2175,28 @@ func (d *ddl) CoalescePartitions(ctx sessionctx.Context, ident ast.Ident, spec * return errors.Trace(ErrPartitionMgmtOnNonpartitioned) } - // Coalesce partition can only be used on hash/key partitions. - if meta.Partition.Type == model.PartitionTypeRange { - return errors.Trace(ErrCoalesceOnlyOnHashPartition) - } - + switch meta.Partition.Type { // We don't support coalesce partitions hash type partition now. - if meta.Partition.Type == model.PartitionTypeHash { + case model.PartitionTypeHash: return errors.Trace(ErrUnsupportedCoalescePartition) + + // Key type partition cannot be constructed currently, ignoring it for now. + case model.PartitionTypeKey: + + // Coalesce partition can only be used on hash/key partitions. + default: + return errors.Trace(ErrCoalesceOnlyOnHashPartition) } return errors.Trace(err) } func (d *ddl) TruncateTablePartition(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { + // TODO: Support truncate multiple partitions + if len(spec.PartitionNames) != 1 { + return errRunMultiSchemaChanges + } + is := d.infoHandle.Get() schema, ok := is.SchemaByName(ident.Schema) if !ok { @@ -2206,7 +2212,7 @@ func (d *ddl) TruncateTablePartition(ctx sessionctx.Context, ident ast.Ident, sp } var pid int64 - pid, err = tables.FindPartitionByName(meta, spec.Name) + pid, err = tables.FindPartitionByName(meta, spec.PartitionNames[0].L) if err != nil { return errors.Trace(err) } @@ -2228,6 +2234,11 @@ func (d *ddl) TruncateTablePartition(ctx sessionctx.Context, ident ast.Ident, sp } func (d *ddl) DropTablePartition(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { + // TODO: Support drop multiple partitions + if len(spec.PartitionNames) != 1 { + return errRunMultiSchemaChanges + } + is := d.infoHandle.Get() schema, ok := is.SchemaByName(ident.Schema) if !ok { @@ -2241,7 +2252,9 @@ func (d *ddl) DropTablePartition(ctx sessionctx.Context, ident ast.Ident, spec * if meta.GetPartitionInfo() == nil { return errors.Trace(ErrPartitionMgmtOnNonpartitioned) } - err = checkDropTablePartition(meta, spec.Name) + + partName := spec.PartitionNames[0].L + err = checkDropTablePartition(meta, partName) if err != nil { return errors.Trace(err) } @@ -2251,7 +2264,7 @@ func (d *ddl) DropTablePartition(ctx sessionctx.Context, ident ast.Ident, spec * TableID: meta.ID, Type: model.ActionDropTablePartition, BinlogInfo: &model.HistoryInfo{}, - Args: []interface{}{spec.Name}, + Args: []interface{}{partName}, } err = d.doDDLJob(ctx, job) @@ -3266,9 +3279,15 @@ func validateCommentLength(vars *variable.SessionVars, comment string, maxLen in } func buildPartitionInfo(meta *model.TableInfo, d *ddl, spec *ast.AlterTableSpec) (*model.PartitionInfo, error) { - if meta.Partition.Type == model.PartitionTypeRange && len(spec.PartDefinitions) == 0 { - return nil, errors.Trace(ErrPartitionsMustBeDefined) + if meta.Partition.Type == model.PartitionTypeRange { + if len(spec.PartDefinitions) == 0 { + return nil, ast.ErrPartitionsMustBeDefined.GenWithStackByArgs(meta.Partition.Type) + } + } else { + // we don't support ADD PARTITION for all other partition types yet. + return nil, errors.Trace(ErrUnsupportedAddPartition) } + part := &model.PartitionInfo{ Type: meta.Partition.Type, Expr: meta.Partition.Expr, @@ -3281,14 +3300,21 @@ func buildPartitionInfo(meta *model.TableInfo, d *ddl, spec *ast.AlterTableSpec) return nil, err } for ith, def := range spec.PartDefinitions { + if err := def.Clause.Validate(part.Type, len(part.Columns)); err != nil { + return nil, err + } + // For RANGE partition only VALUES LESS THAN should be possible. + clause := def.Clause.(*ast.PartitionDefinitionClauseLessThan) + + comment, _ := def.Comment() piDef := model.PartitionDefinition{ Name: def.Name, ID: genIDs[ith], - Comment: def.Comment, + Comment: comment, } buf := new(bytes.Buffer) - for _, expr := range def.LessThan { + for _, expr := range clause.Exprs { expr.Format(buf) piDef.LessThan = append(piDef.LessThan, buf.String()) buf.Reset() diff --git a/ddl/partition.go b/ddl/partition.go index 7686e848312a3..8b66ced4a855c 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -42,6 +42,17 @@ func buildTablePartitionInfo(ctx sessionctx.Context, d *ddl, s *ast.CreateTableS if s.Partition == nil { return nil, nil } + + // force-discard the unsupported types, even when @@tidb_enable_table_partition = 'on' + switch s.Partition.Tp { + case model.PartitionTypeKey: + // can't create a warning for KEY partition, it will fail an integration test :/ + return nil, nil + case model.PartitionTypeList, model.PartitionTypeSystemTime: + ctx.GetSessionVars().StmtCtx.AppendWarning(errUnsupportedCreatePartition) + return nil, nil + } + var enable bool switch ctx.GetSessionVars().EnableTablePartition { case "on": @@ -111,30 +122,34 @@ func buildHashPartitionDefinitions(ctx sessionctx.Context, d *ddl, s *ast.Create defs := make([]model.PartitionDefinition, pi.Num) for i := 0; i < len(defs); i++ { defs[i].ID = genIDs[i] - defs[i].Name = model.NewCIStr(fmt.Sprintf("p%v", i)) + if len(s.Partition.Definitions) == 0 { + defs[i].Name = model.NewCIStr(fmt.Sprintf("p%v", i)) + } else { + def := s.Partition.Definitions[i] + defs[i].Name = def.Name + defs[i].Comment, _ = def.Comment() + } } pi.Definitions = defs return nil } func buildRangePartitionDefinitions(ctx sessionctx.Context, d *ddl, s *ast.CreateTableStmt, pi *model.PartitionInfo) error { - genIDs, err := d.genGlobalIDs(len(s.Partition.Definitions)) + genIDs, err := d.genGlobalIDs(int(pi.Num)) if err != nil { return err } for ith, def := range s.Partition.Definitions { + comment, _ := def.Comment() piDef := model.PartitionDefinition{ Name: def.Name, ID: genIDs[ith], - Comment: def.Comment, + Comment: comment, } - if s.Partition.ColumnNames == nil && len(def.LessThan) != 1 { - return ErrTooManyValues.GenWithStackByArgs(s.Partition.Tp.String()) - } buf := new(bytes.Buffer) // Range columns partitions support multi-column partitions. - for _, expr := range def.LessThan { + for _, expr := range def.Clause.(*ast.PartitionDefinitionClauseLessThan).Exprs { expr.Format(buf) piDef.LessThan = append(piDef.LessThan, buf.String()) buf.Reset() @@ -450,14 +465,14 @@ func checkAddPartitionTooManyPartitions(piDefs uint64) error { func checkNoHashPartitions(ctx sessionctx.Context, partitionNum uint64) error { if partitionNum == 0 { - return ErrNoParts.GenWithStackByArgs("partitions") + return ast.ErrNoParts.GenWithStackByArgs("partitions") } return nil } func checkNoRangePartitions(partitionNum int) error { if partitionNum == 0 { - return errors.Trace(ErrPartitionsMustBeDefined) + return ast.ErrPartitionsMustBeDefined.GenWithStackByArgs("RANGE") } return nil } @@ -558,20 +573,8 @@ func truncateTableByReassignPartitionIDs(t *meta.Meta, tblInfo *model.TableInfo) if err != nil { return errors.Trace(err) } - - var newDef model.PartitionDefinition - if tblInfo.Partition.Type == model.PartitionTypeHash { - newDef = model.PartitionDefinition{ - ID: pid, - } - } else if tblInfo.Partition.Type == model.PartitionTypeRange { - newDef = model.PartitionDefinition{ - ID: pid, - Name: def.Name, - LessThan: def.LessThan, - Comment: def.Comment, - } - } + newDef := def + newDef.ID = pid newDefs = append(newDefs, newDef) } tblInfo.Partition.Definitions = newDefs diff --git a/go.mod b/go.mod index 0fbd9a208bb3a..db40cb81ef737 100644 --- a/go.mod +++ b/go.mod @@ -43,7 +43,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e github.com/pingcap/kvproto v0.0.0-20190703131923-d9830856b531 github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 - github.com/pingcap/parser v0.0.0-20190712081837-c3bdffe5d00e + github.com/pingcap/parser v0.0.0-20190805123000-d4d07337cee2 github.com/pingcap/pd v0.0.0-20190711034019-ee98bf9063e9 github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330 diff --git a/go.sum b/go.sum index 246e269e5d2de..abba4dd6ec78b 100644 --- a/go.sum +++ b/go.sum @@ -167,8 +167,8 @@ github.com/pingcap/kvproto v0.0.0-20190703131923-d9830856b531/go.mod h1:QMdbTAXC github.com/pingcap/log v0.0.0-20190214045112-b37da76f67a7/go.mod h1:xsfkWVaFVV5B8e1K9seWfyJWFrIhbtUTAD8NV1Pq3+w= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 h1:t2OQTpPJnrPDGlvA+3FwJptMTt6MEPdzK1Wt99oaefQ= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw= -github.com/pingcap/parser v0.0.0-20190712081837-c3bdffe5d00e h1:bFfuLDvmMO6QvxkCAEsfJihxBkcrk58MYJRVpM/1Ujk= -github.com/pingcap/parser v0.0.0-20190712081837-c3bdffe5d00e/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= +github.com/pingcap/parser v0.0.0-20190805123000-d4d07337cee2 h1:6xsMMlH8yvLr7rt2yxOjTx8bW3y3OSAXRTEvisTxVS8= +github.com/pingcap/parser v0.0.0-20190805123000-d4d07337cee2/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= github.com/pingcap/pd v0.0.0-20190711034019-ee98bf9063e9 h1:sqqiviE8oEYXJh3Aq59HO/AhxjsvcRb9ETh0ivFOHXc= github.com/pingcap/pd v0.0.0-20190711034019-ee98bf9063e9/go.mod h1:3DlDlFT7EF64A1bmb/tulZb6wbPSagm5G4p1AlhaEDs= github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU=