Skip to content

Commit

Permalink
ddl: add the not expression check when creating a partition table (p…
Browse files Browse the repository at this point in the history
  • Loading branch information
Reminiscent authored and Jianke Tang committed Dec 25, 2020
1 parent 787b255 commit a9a2d2a
Show file tree
Hide file tree
Showing 3 changed files with 75 additions and 31 deletions.
16 changes: 16 additions & 0 deletions ddl/db_partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -505,6 +505,22 @@ create table log_message_1 (
"partition p1 values less than ('G'));",
ddl.ErrRangeNotIncreasing,
},
{
"CREATE TABLE t1(c0 INT) PARTITION BY HASH((NOT c0)) PARTITIONS 2;",
ddl.ErrPartitionFunctionIsNotAllowed,
},
{
"CREATE TABLE t1(c0 INT) PARTITION BY HASH((!c0)) PARTITIONS 2;",
ddl.ErrPartitionFunctionIsNotAllowed,
},
{
"CREATE TABLE t1(c0 INT) PARTITION BY LIST((NOT c0)) (partition p0 values in (0), partition p1 values in (1));",
ddl.ErrPartitionFunctionIsNotAllowed,
},
{
"CREATE TABLE t1(c0 INT) PARTITION BY LIST((!c0)) (partition p0 values in (0), partition p1 values in (1));",
ddl.ErrPartitionFunctionIsNotAllowed,
},
}
for i, t := range cases {
_, err := tk.Exec(t.sql)
Expand Down
45 changes: 14 additions & 31 deletions ddl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@ import (
"github.com/pingcap/parser/format"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/opcode"
"github.com/pingcap/tidb/ddl/placement"
"github.com/pingcap/tidb/ddl/util"
"github.com/pingcap/tidb/domain/infosync"
Expand All @@ -43,6 +42,7 @@ import (
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
driver "github.com/pingcap/tidb/types/parser_driver"
tidbutil "github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/collate"
Expand Down Expand Up @@ -384,7 +384,7 @@ func buildHashPartitionDefinitions(_ sessionctx.Context, defs []*ast.PartitionDe

func buildListPartitionDefinitions(ctx sessionctx.Context, defs []*ast.PartitionDefinition, tbInfo *model.TableInfo) ([]model.PartitionDefinition, error) {
definitions := make([]model.PartitionDefinition, 0, len(defs))
exprChecker := newPartitionExprChecker(ctx, nil, checkPartitionExprAllowed, checkPartitionExprFuncAllowed)
exprChecker := newPartitionExprChecker(ctx, nil, checkPartitionExprAllowed)
for _, def := range defs {
if err := def.Clause.Validate(model.PartitionTypeList, len(tbInfo.Partition.Columns)); err != nil {
return nil, err
Expand Down Expand Up @@ -448,7 +448,7 @@ func collectColumnsType(tbInfo *model.TableInfo) []types.FieldType {

func buildRangePartitionDefinitions(ctx sessionctx.Context, defs []*ast.PartitionDefinition, tbInfo *model.TableInfo) ([]model.PartitionDefinition, error) {
definitions := make([]model.PartitionDefinition, 0, len(defs))
exprChecker := newPartitionExprChecker(ctx, nil, checkPartitionExprAllowed, checkPartitionExprFuncAllowed)
exprChecker := newPartitionExprChecker(ctx, nil, checkPartitionExprAllowed)
for _, def := range defs {
if err := def.Clause.Validate(model.PartitionTypeRange, len(tbInfo.Partition.Columns)); err != nil {
return nil, err
Expand Down Expand Up @@ -595,7 +595,7 @@ func checkPartitionFuncValid(ctx sessionctx.Context, tblInfo *model.TableInfo, e
if expr == nil {
return nil
}
exprChecker := newPartitionExprChecker(ctx, tblInfo, checkPartitionExprArgs, checkPartitionExprAllowed, checkPartitionExprFuncAllowed)
exprChecker := newPartitionExprChecker(ctx, tblInfo, checkPartitionExprArgs, checkPartitionExprAllowed)
expr.Accept(exprChecker)
if exprChecker.err != nil {
return errors.Trace(exprChecker.err)
Expand Down Expand Up @@ -1707,37 +1707,20 @@ func (p *partitionExprChecker) extractColumns(_ sessionctx.Context, _ *model.Tab

func checkPartitionExprAllowed(_ sessionctx.Context, _ *model.TableInfo, e ast.ExprNode) error {
switch v := e.(type) {
case *ast.FuncCastExpr, *ast.CaseExpr, *ast.SubqueryExpr, *ast.WindowFuncExpr, *ast.RowExpr, *ast.DefaultExpr, *ast.ValuesExpr,
*ast.SetCollationExpr:
return errors.Trace(ErrPartitionFunctionIsNotAllowed)
case *ast.FuncCallExpr:
if _, ok := expression.AllowedPartitionFuncMap[v.FnName.L]; ok {
return nil
}
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
switch v.Op {
case opcode.Or, opcode.And, opcode.Xor, opcode.LeftShift, opcode.RightShift, opcode.BitNeg, opcode.Div:
return errors.Trace(ErrPartitionFunctionIsNotAllowed)
if _, ok := expression.AllowedPartition4BinaryOpMap[v.Op]; ok {
return nil
}
case *ast.UnaryOperationExpr:
if v.Op == opcode.BitNeg {
return errors.Trace(ErrPartitionFunctionIsNotAllowed)
if _, ok := expression.AllowedPartition4UnaryOpMap[v.Op]; ok {
return nil
}
}
return nil
}

func checkPartitionExprFuncAllowed(_ sessionctx.Context, _ *model.TableInfo, e ast.ExprNode) error {
expr, ok := e.(*ast.FuncCallExpr)
if !ok {
return nil
}
allowedFuncMap := map[string]struct{}{
ast.ToDays: {}, ast.ToSeconds: {}, ast.DayOfMonth: {}, ast.Month: {}, ast.DayOfYear: {},
ast.Quarter: {}, ast.YearWeek: {}, ast.Year: {}, ast.Weekday: {}, ast.DayOfWeek: {}, ast.Day: {},
ast.Hour: {}, ast.Minute: {}, ast.Second: {}, ast.TimeToSec: {}, ast.MicroSecond: {},
ast.UnixTimestamp: {}, ast.FromDays: {}, ast.Extract: {}, ast.Abs: {}, ast.Ceiling: {},
ast.DateDiff: {}, ast.Floor: {}, ast.Mod: {},
}
if _, ok := allowedFuncMap[expr.FnName.L]; ok {
case *ast.ColumnNameExpr, *ast.ParenthesesExpr, *driver.ValueExpr, *ast.MaxValueExpr,
*ast.TimeUnitExpr:
return nil
}
return errors.Trace(ErrPartitionFunctionIsNotAllowed)
Expand Down
45 changes: 45 additions & 0 deletions expression/function_traits.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ package expression

import (
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/opcode"
)

// UnCacheableFunctions stores functions which can not be cached to plan cache.
Expand Down Expand Up @@ -133,6 +134,50 @@ var DeferredFunctions = map[string]struct{}{
ast.UTCDate: {},
}

// AllowedPartitionFuncMap stores functions which can be used in the partition expression.
var AllowedPartitionFuncMap = map[string]struct{}{
ast.ToDays: {},
ast.ToSeconds: {},
ast.DayOfMonth: {},
ast.Month: {},
ast.DayOfYear: {},
ast.Quarter: {},
ast.YearWeek: {},
ast.Year: {},
ast.Weekday: {},
ast.DayOfWeek: {},
ast.Day: {},
ast.Hour: {},
ast.Minute: {},
ast.Second: {},
ast.TimeToSec: {},
ast.MicroSecond: {},
ast.UnixTimestamp: {},
ast.FromDays: {},
ast.Extract: {},
ast.Abs: {},
ast.Ceiling: {},
ast.DateDiff: {},
ast.Floor: {},
ast.Mod: {},
}

// AllowedPartition4BinaryOpMap store the operator for Binary Expr
// See https://dev.mysql.com/doc/refman/5.7/en/partitioning-limitations.html for more details
var AllowedPartition4BinaryOpMap = map[opcode.Op]struct{}{
opcode.Plus: {},
opcode.Minus: {},
opcode.Mul: {},
opcode.IntDiv: {},
opcode.Mod: {},
}

// AllowedPartition4UnaryOpMap store the operator for Unary Expr
var AllowedPartition4UnaryOpMap = map[opcode.Op]struct{}{
opcode.Plus: {},
opcode.Minus: {},
}

// inequalFunctions stores functions which cannot be propagated from column equal condition.
var inequalFunctions = map[string]struct{}{
ast.IsNull: {},
Expand Down

0 comments on commit a9a2d2a

Please sign in to comment.