Skip to content

Commit

Permalink
Merge branch 'master' into master-3.0-issue
Browse files Browse the repository at this point in the history
  • Loading branch information
lovewin99 authored Jul 8, 2019
2 parents 2f67f9d + fdbc149 commit 0aaf491
Show file tree
Hide file tree
Showing 7 changed files with 311 additions and 32 deletions.
2 changes: 1 addition & 1 deletion config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -597,7 +597,7 @@ func hasRootPrivilege() bool {
}

// TableLockEnabled uses to check whether enabled the table lock feature.
var TableLockEnabled = func() bool {
func TableLockEnabled() bool {
return GetGlobalConfig().EnableTableLock
}

Expand Down
118 changes: 118 additions & 0 deletions ddl/db_partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1474,6 +1474,124 @@ func (s *testIntegrationSuite3) TestPartitionErrorCode(c *C) {
c.Assert(ddl.ErrCoalesceOnlyOnHashPartition.Equal(err), IsTrue)
}

func (s *testIntegrationSuite5) TestConstAndTimezoneDepent(c *C) {
tk := testkit.NewTestKit(c, s.store)
// add partition
tk.MustExec("set @@session.tidb_enable_table_partition = 1")
tk.MustExec("drop database if exists test_db_with_partition_const")
tk.MustExec("create database test_db_with_partition_const")
tk.MustExec("use test_db_with_partition_const")

sql1 := `create table t1 ( id int )
partition by range(4) (
partition p1 values less than (10)
);`
assertErrorCode(c, tk, sql1, tmysql.ErrWrongExprInPartitionFunc)

sql2 := `create table t2 ( time_recorded timestamp )
partition by range(TO_DAYS(time_recorded)) (
partition p1 values less than (1559192604)
);`
assertErrorCode(c, tk, sql2, tmysql.ErrWrongExprInPartitionFunc)

sql3 := `create table t3 ( id int )
partition by range(DAY(id)) (
partition p1 values less than (1)
);`
assertErrorCode(c, tk, sql3, tmysql.ErrWrongExprInPartitionFunc)

sql4 := `create table t4 ( id int )
partition by hash(4) partitions 4
;`
assertErrorCode(c, tk, sql4, tmysql.ErrWrongExprInPartitionFunc)

sql5 := `create table t5 ( time_recorded timestamp )
partition by range(to_seconds(time_recorded)) (
partition p1 values less than (1559192604)
);`
assertErrorCode(c, tk, sql5, tmysql.ErrWrongExprInPartitionFunc)

sql6 := `create table t6 ( id int )
partition by range(to_seconds(id)) (
partition p1 values less than (1559192604)
);`
assertErrorCode(c, tk, sql6, tmysql.ErrWrongExprInPartitionFunc)

sql7 := `create table t7 ( time_recorded timestamp )
partition by range(abs(time_recorded)) (
partition p1 values less than (1559192604)
);`
assertErrorCode(c, tk, sql7, tmysql.ErrWrongExprInPartitionFunc)

sql8 := `create table t2332 ( time_recorded time )
partition by range(TO_DAYS(time_recorded)) (
partition p0 values less than (1)
);`
assertErrorCode(c, tk, sql8, tmysql.ErrWrongExprInPartitionFunc)

sql9 := `create table t1 ( id int )
partition by hash(4) partitions 4;`
assertErrorCode(c, tk, sql9, tmysql.ErrWrongExprInPartitionFunc)

sql10 := `create table t1 ( id int )
partition by hash(ed) partitions 4;`
assertErrorCode(c, tk, sql10, tmysql.ErrBadField)

sql11 := `create table t2332 ( time_recorded time )
partition by range(TO_SECONDS(time_recorded)) (
partition p0 values less than (1)
);`
assertErrorCode(c, tk, sql11, tmysql.ErrWrongExprInPartitionFunc)

sql12 := `create table t2332 ( time_recorded time )
partition by range(TO_SECONDS(time_recorded)) (
partition p0 values less than (1)
);`
assertErrorCode(c, tk, sql12, tmysql.ErrWrongExprInPartitionFunc)

sql13 := `create table t2332 ( time_recorded time )
partition by range(day(time_recorded)) (
partition p0 values less than (1)
);`
assertErrorCode(c, tk, sql13, tmysql.ErrWrongExprInPartitionFunc)

sql14 := `create table t2332 ( time_recorded timestamp )
partition by range(day(time_recorded)) (
partition p0 values less than (1)
);`
assertErrorCode(c, tk, sql14, tmysql.ErrWrongExprInPartitionFunc)
}

func (s *testIntegrationSuite5) TestConstAndTimezoneDepent2(c *C) {
tk := testkit.NewTestKit(c, s.store)
// add partition
tk.MustExec("set @@session.tidb_enable_table_partition = 1")
tk.MustExec("drop database if exists test_db_with_partition_const")
tk.MustExec("create database test_db_with_partition_const")
tk.MustExec("use test_db_with_partition_const")

tk.MustExec(`create table t1 ( time_recorded datetime )
partition by range(TO_DAYS(time_recorded)) (
partition p0 values less than (1));`)

tk.MustExec(`create table t2 ( time_recorded date )
partition by range(TO_DAYS(time_recorded)) (
partition p0 values less than (1));`)

tk.MustExec(`create table t3 ( time_recorded date )
partition by range(TO_SECONDS(time_recorded)) (
partition p0 values less than (1));`)

tk.MustExec(`create table t4 ( time_recorded date )
partition by range(TO_SECONDS(time_recorded)) (
partition p0 values less than (1));`)

tk.MustExec(`create table t5 ( time_recorded timestamp )
partition by range(unix_timestamp(time_recorded)) (
partition p1 values less than (1559192604)
);`)
}

func (s *testIntegrationSuite3) TestUnsupportedPartitionManagementDDLs(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test;")
Expand Down
13 changes: 4 additions & 9 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -81,14 +81,6 @@ type testDBSuite struct {
autoIDStep int64
}

var tableLockEnabled uint32 = 0

func init() {
config.TableLockEnabled = func() bool {
return atomic.LoadUint32(&tableLockEnabled) > 0
}
}

func setUpSuite(s *testDBSuite, c *C) {
var err error

Expand All @@ -99,7 +91,10 @@ func setUpSuite(s *testDBSuite, c *C) {
s.autoIDStep = autoid.GetStep()
ddl.WaitTimeWhenErrorOccured = 0
// Test for table lock.
atomic.StoreUint32(&tableLockEnabled, 1)
cfg := config.GetGlobalConfig()
newCfg := *cfg
newCfg.EnableTableLock = true
config.StoreGlobalConfig(&newCfg)

s.cluster = mocktikv.NewCluster()
mocktikv.BootstrapWithSingleStore(s.cluster)
Expand Down
128 changes: 113 additions & 15 deletions ddl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -177,6 +177,70 @@ func checkPartitionNameUnique(tbInfo *model.TableInfo, pi *model.PartitionInfo)
return nil
}

// See https://github.com/mysql/mysql-server/blob/5.7/sql/item_func.h#L387
func hasTimestampField(ctx sessionctx.Context, tblInfo *model.TableInfo, expr ast.ExprNode) (bool, error) {
partCols, err := partitionColumns(ctx, tblInfo, expr)
if err != nil {
return false, err
}

for _, c := range partCols {
if c.GetType().Tp == mysql.TypeTimestamp {
return true, nil
}
}

return false, nil
}

// See https://github.com/mysql/mysql-server/blob/5.7/sql/item_func.h#L399
func hasDateField(ctx sessionctx.Context, tblInfo *model.TableInfo, expr ast.ExprNode) (bool, error) {
partCols, err := partitionColumns(ctx, tblInfo, expr)
if err != nil {
return false, err
}

for _, c := range partCols {
if c.GetType().Tp == mysql.TypeDate || c.GetType().Tp == mysql.TypeDatetime {
return true, nil
}
}

return false, nil
}

// See https://github.com/mysql/mysql-server/blob/5.7/sql/item_func.h#L412
func hasTimeField(ctx sessionctx.Context, tblInfo *model.TableInfo, expr ast.ExprNode) (bool, error) {
partCols, err := partitionColumns(ctx, tblInfo, expr)
if err != nil {
return false, err
}

for _, c := range partCols {
if c.GetType().Tp == mysql.TypeDatetime || c.GetType().Tp == mysql.TypeDuration {
return true, nil
}
}

return false, nil
}

// We assume the result of any function that has a TIMESTAMP argument to be
// timezone-dependent, since a TIMESTAMP value in both numeric and string
// contexts is interpreted according to the current timezone.
// The only exception is UNIX_TIMESTAMP() which returns the internal
// representation of a TIMESTAMP argument verbatim, and thus does not depend on
// the timezone.
// See https://github.com/mysql/mysql-server/blob/5.7/sql/item_func.h#L445
func defaultTimezoneDependent(ctx sessionctx.Context, tblInfo *model.TableInfo, expr ast.ExprNode) (bool, error) {
v, err := hasTimestampField(ctx, tblInfo, expr)
if err != nil {
return false, err
}

return !v, nil
}

// checkPartitionFuncValid checks partition function validly.
func checkPartitionFuncValid(ctx sessionctx.Context, tblInfo *model.TableInfo, expr ast.ExprNode) error {
switch v := expr.(type) {
Expand All @@ -186,23 +250,23 @@ func checkPartitionFuncValid(ctx sessionctx.Context, tblInfo *model.TableInfo, e
// check function which allowed in partitioning expressions
// see https://dev.mysql.com/doc/mysql-partitioning-excerpt/5.7/en/partitioning-limitations-functions.html
switch v.FnName.L {
case ast.Abs, ast.Ceiling, ast.DateDiff, ast.Day, ast.DayOfMonth, ast.DayOfWeek, ast.DayOfYear, ast.Extract, ast.Floor,
ast.Hour, ast.MicroSecond, ast.Minute, ast.Mod, ast.Month, ast.Quarter, ast.Second, ast.TimeToSec, ast.ToDays,
ast.ToSeconds, ast.Weekday, ast.Year, ast.YearWeek:
return nil
// Mysql don't allow creating partitions with expressions with non matching
// arguments as a (sub)partitioning function,
// but we want to allow such expressions when opening existing tables for
// easier maintenance. This exception should be deprecated at some point in future so that we always throw an error.
// See https://github.com/mysql/mysql-server/blob/5.7/sql/sql_partition.cc#L1072
case ast.Day, ast.DayOfMonth, ast.DayOfWeek, ast.DayOfYear, ast.Month, ast.Quarter, ast.ToDays, ast.ToSeconds,
ast.Weekday, ast.Year, ast.YearWeek:
return checkPartitionFunc(hasDateField(ctx, tblInfo, expr))
case ast.Hour, ast.MicroSecond, ast.Minute, ast.Second, ast.TimeToSec:
return checkPartitionFunc(hasTimeField(ctx, tblInfo, expr))
case ast.UnixTimestamp:
if len(v.Args) == 1 {
col, err := expression.RewriteSimpleExprWithTableInfo(ctx, tblInfo, v.Args[0])
if err != nil {
return errors.Trace(err)
}
if col.GetType().Tp != mysql.TypeTimestamp {
return errors.Trace(errWrongExprInPartitionFunc)
}
return nil
}
return checkPartitionFunc(hasTimestampField(ctx, tblInfo, expr))
case ast.Abs, ast.Ceiling, ast.DateDiff, ast.Extract, ast.Floor, ast.Mod:
return checkPartitionFunc(defaultTimezoneDependent(ctx, tblInfo, expr))
default:
return errors.Trace(ErrPartitionFunctionIsNotAllowed)
}
return errors.Trace(ErrPartitionFunctionIsNotAllowed)
case *ast.BinaryOperationExpr:
// The DIV operator (opcode.IntDiv) is also supported; the / operator ( opcode.Div ) is not permitted.
// see https://dev.mysql.com/doc/refman/5.7/en/partitioning-limitations.html
Expand All @@ -215,10 +279,44 @@ func checkPartitionFuncValid(ctx sessionctx.Context, tblInfo *model.TableInfo, e
if v.Op == opcode.BitNeg {
return errors.Trace(ErrPartitionFunctionIsNotAllowed)
}
return nil
}

// check constant.
_, err := partitionColumns(ctx, tblInfo, expr)
return err
}

// For partition tables, mysql do not support Constant, random or timezone-dependent expressions
// Based on mysql code to check whether field is valid, every time related type has check_valid_arguments_processor function.
// See https://github.com/mysql/mysql-server/blob/5.7/sql/item_timefunc.
func checkPartitionFunc(isTimezoneDependent bool, err error) error {
if err != nil {
return err
}

if !isTimezoneDependent {
return errors.Trace(errWrongExprInPartitionFunc)
}

return nil
}

func partitionColumns(ctx sessionctx.Context, tblInfo *model.TableInfo, expr ast.ExprNode) ([]*expression.Column, error) {
buf := new(bytes.Buffer)
expr.Format(buf)
partCols, err := extractPartitionColumns(ctx, buf.String(), tblInfo)
if err != nil {
return nil, errors.Trace(err)
}

if len(partCols) == 0 {
return nil, errors.Trace(errWrongExprInPartitionFunc)
}

return partCols, nil
}

// checkPartitionFuncType checks partition function return type.
func checkPartitionFuncType(ctx sessionctx.Context, s *ast.CreateTableStmt, cols []*table.Column, tblInfo *model.TableInfo) error {
if s.Partition.Expr == nil {
Expand Down
14 changes: 8 additions & 6 deletions ddl/serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,14 +18,14 @@ import (
"fmt"
"strings"
"sync"
"sync/atomic"
"time"

. "github.com/pingcap/check"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/ddl"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/infoschema"
Expand Down Expand Up @@ -418,14 +418,16 @@ func (s *testSerialSuite) TestTableLocksEnable(c *C) {
tk.MustExec("drop table if exists t1")
defer tk.MustExec("drop table if exists t1")
tk.MustExec("create table t1 (a int)")
// recover table lock config.
originValue := atomic.LoadUint32(&tableLockEnabled)

// Test for enable table lock config.
cfg := config.GetGlobalConfig()
newCfg := *cfg
newCfg.EnableTableLock = false
config.StoreGlobalConfig(&newCfg)
defer func() {
atomic.StoreUint32(&tableLockEnabled, originValue)
config.StoreGlobalConfig(cfg)
}()

// Test for enable table lock config.
atomic.StoreUint32(&tableLockEnabled, 0)
tk.MustExec("lock tables t1 write")
checkTableLock(c, tk.Se, "test", "t1", model.TableLockNone)
}
Loading

0 comments on commit 0aaf491

Please sign in to comment.