Skip to content

Commit

Permalink
Merge branch 'master' into corcol-bug
Browse files Browse the repository at this point in the history
  • Loading branch information
zz-jason authored Aug 24, 2018
2 parents b5bfe43 + db873d6 commit 1ea1dc1
Show file tree
Hide file tree
Showing 25 changed files with 305 additions and 102 deletions.
5 changes: 5 additions & 0 deletions .gitattributes
Original file line number Diff line number Diff line change
@@ -0,0 +1,5 @@
# Set the default behavior, in case people don't have core.autocrlf set.
* text=auto

# Declare files that will always have LF line endings on checkout.
*.y text eol=lf
3 changes: 3 additions & 0 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -188,6 +188,7 @@ var (
ErrPartitionFuncNotAllowed = terror.ClassDDL.New(codeErrPartitionFuncNotAllowed, mysql.MySQLErrName[mysql.ErrPartitionFuncNotAllowed])
// ErrUniqueKeyNeedAllFieldsInPf returns must include all columns in the table's partitioning function.
ErrUniqueKeyNeedAllFieldsInPf = terror.ClassDDL.New(codeUniqueKeyNeedAllFieldsInPf, mysql.MySQLErrName[mysql.ErrUniqueKeyNeedAllFieldsInPf])
errWrongExprInPartitionFunc = terror.ClassDDL.New(codeWrongExprInPartitionFunc, mysql.MySQLErrName[mysql.ErrWrongExprInPartitionFunc])
)

// DDL is responsible for updating schema in data store and maintaining in-memory InfoSchema cache.
Expand Down Expand Up @@ -606,6 +607,7 @@ const (
codeErrFieldTypeNotAllowedAsPartitionField = terror.ErrCode(mysql.ErrFieldTypeNotAllowedAsPartitionField)
codeUniqueKeyNeedAllFieldsInPf = terror.ErrCode(mysql.ErrUniqueKeyNeedAllFieldsInPf)
codePrimaryCantHaveNull = terror.ErrCode(mysql.ErrPrimaryCantHaveNull)
codeWrongExprInPartitionFunc = terror.ErrCode(mysql.ErrWrongExprInPartitionFunc)
)

func init() {
Expand Down Expand Up @@ -652,6 +654,7 @@ func init() {
codeErrFieldTypeNotAllowedAsPartitionField: mysql.ErrFieldTypeNotAllowedAsPartitionField,
codeUniqueKeyNeedAllFieldsInPf: mysql.ErrUniqueKeyNeedAllFieldsInPf,
codePrimaryCantHaveNull: mysql.ErrPrimaryCantHaveNull,
codeWrongExprInPartitionFunc: mysql.ErrWrongExprInPartitionFunc,
}
terror.ErrClassToMySQLCodes[terror.ClassDDL] = ddlMySQLErrCodes
}
2 changes: 1 addition & 1 deletion ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -912,7 +912,7 @@ func (d *ddl) CreateTable(ctx sessionctx.Context, s *ast.CreateTableStmt) (err e
return errors.Trace(err)
}

if err = checkPartitionFuncValid(s.Partition.Expr); err != nil {
if err = checkPartitionFuncValid(ctx, tbInfo, s.Partition.Expr); err != nil {
return errors.Trace(err)
}

Expand Down
32 changes: 24 additions & 8 deletions ddl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/parser/opcode"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/table"
Expand Down Expand Up @@ -103,28 +104,43 @@ func checkPartitionNameUnique(tbInfo *model.TableInfo, pi *model.PartitionInfo)
}

// checkPartitionFuncValid checks partition function validly.
func checkPartitionFuncValid(expr ast.ExprNode) error {
func checkPartitionFuncValid(ctx sessionctx.Context, tblInfo *model.TableInfo, expr ast.ExprNode) error {
switch v := expr.(type) {
case *ast.CaseExpr:
return ErrPartitionFunctionIsNotAllowed
case *ast.FuncCastExpr, *ast.CaseExpr:
return errors.Trace(ErrPartitionFunctionIsNotAllowed)
case *ast.FuncCallExpr:
// 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.UnixTimestamp, ast.Weekday, ast.Year, ast.YearWeek:
ast.ToSeconds, ast.Weekday, ast.Year, ast.YearWeek:
return nil
default:
return ErrPartitionFunctionIsNotAllowed
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 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
if v.Op == opcode.Div {
return ErrPartitionFunctionIsNotAllowed
switch v.Op {
case opcode.Or, opcode.And, opcode.Xor, opcode.LeftShift, opcode.RightShift, opcode.BitNeg, opcode.Div:
return errors.Trace(ErrPartitionFunctionIsNotAllowed)
}
return nil
case *ast.UnaryOperationExpr:
if v.Op == opcode.BitNeg {
return errors.Trace(ErrPartitionFunctionIsNotAllowed)
}
}
return nil
}
Expand Down
1 change: 1 addition & 0 deletions docs/design/2018-07-01-refactor-aggregate-framework.md
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ The `AggFunc` interface contains the following functions:
- `ResetPartialResult` resets the partial result to the original state for a specific aggregate function. It converts the input `PartialResult` to the specific data structure which stores the partial result and then resets every field to the proper original state.
- `UpdatePartialResult` updates the specific partial result for an aggregate function using the input rows which all belong to the same data group. It converts the `PartialResult` to the specific data structure which stores the partial result and then iterates on the input rows and updates that partial result according to the functionality and the state of the aggregate function.
- `AppendFinalResult2Chunk` finalizes the partial result and appends the final result directly to the input `Chunk`. Like other operations, it converts the input `PartialResult` to the specific data structure firstly, calculates the final result and appends that final result to the `Chunk` provided.
- `MergePartialResult` evaluates the final result using the input `PartialResults`. Suppose the input `PartialResults` names are `dst` and `src` respectively. It converts `dst` and `src` to the same data structure firstly, merges the partial results and store the result in `dst`.

The new framework uses the `Build()` function to build an executable aggregate function. Its input parameters are:
- `aggFuncDesc`: the aggregate function representation used by the planner layer.
Expand Down
5 changes: 3 additions & 2 deletions docs/design/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -4,8 +4,9 @@ A proposal template: [TEMPLATE.md](./TEMPLATE.md)

## Proposed

## In Progress

- [Proposal: A new aggregate function execution framework](./2018-07-01-refactor-aggregate-framework.md)
- [Proposal: A new storage row format for efficient decoding](./2018-07-19-row-format.md)

## In Progress

## Completed
59 changes: 31 additions & 28 deletions expression/builtin_cast.go
Original file line number Diff line number Diff line change
Expand Up @@ -1633,11 +1633,13 @@ func (i inCastContext) String() string {
return "__cast_ctx"
}

// inUnionCastContext is session key value that indicates whether executing in union cast context.
// inUnionCastContext is session key value that indicates whether executing in
// union cast context.
// @see BuildCastFunction4Union
const inUnionCastContext inCastContext = 0

// BuildCastFunction4Union build a implicitly CAST ScalarFunction from the Union Expression.
// BuildCastFunction4Union build a implicitly CAST ScalarFunction from the Union
// Expression.
func BuildCastFunction4Union(ctx sessionctx.Context, expr Expression, tp *types.FieldType) (res Expression) {
ctx.SetValue(inUnionCastContext, struct{}{})
defer func() {
Expand Down Expand Up @@ -1673,17 +1675,16 @@ func BuildCastFunction(ctx sessionctx.Context, expr Expression, tp *types.FieldT
Function: f,
}
// We do not fold CAST if the eval type of this scalar function is ETJson
// since we may reset the flag of the field type of CastAsJson later which would
// affect the evaluation of it.
// since we may reset the flag of the field type of CastAsJson later which
// would affect the evaluation of it.
if tp.EvalType() != types.ETJson {
res = FoldConstant(res)
}
return res
}

// WrapWithCastAsInt wraps `expr` with `cast` if the return type
// of expr is not type int,
// otherwise, returns `expr` directly.
// WrapWithCastAsInt wraps `expr` with `cast` if the return type of expr is not
// type int, otherwise, returns `expr` directly.
func WrapWithCastAsInt(ctx sessionctx.Context, expr Expression) Expression {
if expr.GetType().EvalType() == types.ETInt {
return expr
Expand All @@ -1695,9 +1696,8 @@ func WrapWithCastAsInt(ctx sessionctx.Context, expr Expression) Expression {
return BuildCastFunction(ctx, expr, tp)
}

// WrapWithCastAsReal wraps `expr` with `cast` if the return type
// of expr is not type real,
// otherwise, returns `expr` directly.
// WrapWithCastAsReal wraps `expr` with `cast` if the return type of expr is not
// type real, otherwise, returns `expr` directly.
func WrapWithCastAsReal(ctx sessionctx.Context, expr Expression) Expression {
if expr.GetType().EvalType() == types.ETReal {
return expr
Expand All @@ -1709,9 +1709,8 @@ func WrapWithCastAsReal(ctx sessionctx.Context, expr Expression) Expression {
return BuildCastFunction(ctx, expr, tp)
}

// WrapWithCastAsDecimal wraps `expr` with `cast` if the return type
// of expr is not type decimal,
// otherwise, returns `expr` directly.
// WrapWithCastAsDecimal wraps `expr` with `cast` if the return type of expr is
// not type decimal, otherwise, returns `expr` directly.
func WrapWithCastAsDecimal(ctx sessionctx.Context, expr Expression) Expression {
if expr.GetType().EvalType() == types.ETDecimal {
return expr
Expand All @@ -1723,15 +1722,22 @@ func WrapWithCastAsDecimal(ctx sessionctx.Context, expr Expression) Expression {
return BuildCastFunction(ctx, expr, tp)
}

// WrapWithCastAsString wraps `expr` with `cast` if the return type
// of expr is not type string,
// otherwise, returns `expr` directly.
// WrapWithCastAsString wraps `expr` with `cast` if the return type of expr is
// not type string, otherwise, returns `expr` directly.
func WrapWithCastAsString(ctx sessionctx.Context, expr Expression) Expression {
if expr.GetType().EvalType() == types.ETString {
exprTp := expr.GetType()
if exprTp.EvalType() == types.ETString {
return expr
}
argLen := expr.GetType().Flen
if expr.GetType().EvalType() == types.ETInt {
argLen := exprTp.Flen
// If expr is decimal, we should take the decimal point and negative sign
// into consideration, so we set `expr.GetType().Flen + 2` as the `argLen`.
// Since the length of float and double is not accurate, we do not handle
// them.
if exprTp.Tp == mysql.TypeNewDecimal && argLen != types.UnspecifiedFsp {
argLen += 2
}
if exprTp.EvalType() == types.ETInt {
argLen = mysql.MaxIntWidth
}
tp := types.NewFieldType(mysql.TypeVarString)
Expand All @@ -1740,9 +1746,8 @@ func WrapWithCastAsString(ctx sessionctx.Context, expr Expression) Expression {
return BuildCastFunction(ctx, expr, tp)
}

// WrapWithCastAsTime wraps `expr` with `cast` if the return type
// of expr is not same as type of the specified `tp` ,
// otherwise, returns `expr` directly.
// WrapWithCastAsTime wraps `expr` with `cast` if the return type of expr is not
// same as type of the specified `tp` , otherwise, returns `expr` directly.
func WrapWithCastAsTime(ctx sessionctx.Context, expr Expression, tp *types.FieldType) Expression {
exprTp := expr.GetType().Tp
if tp.Tp == exprTp {
Expand All @@ -1769,9 +1774,8 @@ func WrapWithCastAsTime(ctx sessionctx.Context, expr Expression, tp *types.Field
return BuildCastFunction(ctx, expr, tp)
}

// WrapWithCastAsDuration wraps `expr` with `cast` if the return type
// of expr is not type duration,
// otherwise, returns `expr` directly.
// WrapWithCastAsDuration wraps `expr` with `cast` if the return type of expr is
// not type duration, otherwise, returns `expr` directly.
func WrapWithCastAsDuration(ctx sessionctx.Context, expr Expression) Expression {
if expr.GetType().Tp == mysql.TypeDuration {
return expr
Expand All @@ -1790,9 +1794,8 @@ func WrapWithCastAsDuration(ctx sessionctx.Context, expr Expression) Expression
return BuildCastFunction(ctx, expr, tp)
}

// WrapWithCastAsJSON wraps `expr` with `cast` if the return type
// of expr is not type json,
// otherwise, returns `expr` directly.
// WrapWithCastAsJSON wraps `expr` with `cast` if the return type of expr is not
// type json, otherwise, returns `expr` directly.
func WrapWithCastAsJSON(ctx sessionctx.Context, expr Expression) Expression {
if expr.GetType().Tp == mysql.TypeJSON && !mysql.HasParseToJSONFlag(expr.GetType().Flag) {
return expr
Expand Down
2 changes: 1 addition & 1 deletion expression/builtin_time.go
Original file line number Diff line number Diff line change
Expand Up @@ -5410,7 +5410,7 @@ func (b *builtinLastDaySig) evalTime(row chunk.Row) (types.Time, bool, error) {
// getExpressionFsp calculates the fsp from given expression.
func getExpressionFsp(ctx sessionctx.Context, expression Expression) (int, error) {
constExp, isConstant := expression.(*Constant)
if isConstant && types.IsString(expression.GetType()) && !isTemporalColumn(expression) {
if isConstant && types.IsString(expression.GetType().Tp) && !isTemporalColumn(expression) {
str, isNil, err := constExp.EvalString(ctx, chunk.Row{})
if isNil || err != nil {
return 0, errors.Trace(err)
Expand Down
7 changes: 7 additions & 0 deletions expression/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -925,6 +925,13 @@ func (s *testIntegrationSuite) TestStringBuiltin(c *C) {
result.Check(testkit.Rows("2 0 3 0"))
result = tk.MustQuery(`select field("abc", "a", 1), field(1.3, "1.3", 1.5);`)
result.Check(testkit.Rows("1 1"))

tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a decimal(11, 8), b decimal(11,8))")
tk.MustExec("insert into t values('114.57011441','38.04620115'), ('-38.04620119', '38.04620115');")
result = tk.MustQuery("select a,b,concat_ws(',',a,b) from t")
result.Check(testkit.Rows("114.57011441 38.04620115 114.57011441,38.04620115",
"-38.04620119 38.04620115 -38.04620119,38.04620115"))
}

func (s *testIntegrationSuite) TestEncryptionBuiltin(c *C) {
Expand Down
2 changes: 1 addition & 1 deletion expression/typeinfer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -403,7 +403,7 @@ func (s *testInferTypeSuite) createTestCase4StrFuncs() []typeInferTestCase {
{"reverse(c_bigint_d )", mysql.TypeVarString, charset.CharsetUTF8, 0, 20, types.UnspecifiedLength},
{"reverse(c_float_d )", mysql.TypeVarString, charset.CharsetUTF8, 0, 12, types.UnspecifiedLength},
{"reverse(c_double_d )", mysql.TypeVarString, charset.CharsetUTF8, 0, 22, types.UnspecifiedLength},
{"reverse(c_decimal )", mysql.TypeVarString, charset.CharsetUTF8, 0, 6, types.UnspecifiedLength},
{"reverse(c_decimal )", mysql.TypeVarString, charset.CharsetUTF8, 0, 8, types.UnspecifiedLength},
{"reverse(c_char )", mysql.TypeVarString, charset.CharsetUTF8, 0, 20, types.UnspecifiedLength},
{"reverse(c_varchar )", mysql.TypeVarString, charset.CharsetUTF8, 0, 20, types.UnspecifiedLength},
{"reverse(c_text_d )", mysql.TypeVarString, charset.CharsetUTF8, 0, 65535, types.UnspecifiedLength},
Expand Down
47 changes: 30 additions & 17 deletions infoschema/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -792,8 +792,12 @@ func dataForColumns(ctx sessionctx.Context, schemas []*model.DBInfo) [][]types.D
func dataForColumnsInTable(schema *model.DBInfo, tbl *model.TableInfo) [][]types.Datum {
var rows [][]types.Datum
for i, col := range tbl.Columns {
var charMaxLen, charOctLen, numericPrecision, numericScale, datetimePrecision interface{}
colLen, decimal := col.Flen, col.Decimal
defaultFlen, defaultDecimal := mysql.GetDefaultFieldLengthAndDecimal(col.Tp)
if decimal == types.UnspecifiedLength {
decimal = defaultDecimal
}
if colLen == types.UnspecifiedLength {
colLen = defaultFlen
}
Expand All @@ -808,6 +812,8 @@ func dataForColumnsInTable(schema *model.DBInfo, tbl *model.TableInfo) [][]types
if len(col.Elems) != 0 {
colLen += (len(col.Elems) - 1)
}
charMaxLen = colLen
charOctLen = colLen
} else if col.Tp == mysql.TypeEnum {
// Example: In MySQL enum('a', 'ab', 'cdef') has length 4, because
// the longest string in the enum is 'cdef'
Expand All @@ -818,9 +824,16 @@ func dataForColumnsInTable(schema *model.DBInfo, tbl *model.TableInfo) [][]types
colLen = len(ele)
}
}
}
if decimal == types.UnspecifiedLength {
decimal = defaultDecimal
charMaxLen = colLen
charOctLen = colLen
} else if types.IsString(col.Tp) {
charMaxLen = colLen
charOctLen = colLen
} else if types.IsTypeFractionable(col.Tp) {
datetimePrecision = decimal
} else if types.IsTypeNumeric(col.Tp) {
numericPrecision = colLen
numericScale = decimal
}
columnType := col.FieldType.InfoSchemaStr()
columnDesc := table.NewColDesc(table.ToColumn(col))
Expand All @@ -837,19 +850,19 @@ func dataForColumnsInTable(schema *model.DBInfo, tbl *model.TableInfo) [][]types
columnDefault, // COLUMN_DEFAULT
columnDesc.Null, // IS_NULLABLE
types.TypeToStr(col.Tp, col.Charset), // DATA_TYPE
colLen, // CHARACTER_MAXIMUM_LENGTH
colLen, // CHARACTER_OCTET_LENGTH
decimal, // NUMERIC_PRECISION
0, // NUMERIC_SCALE
0, // DATETIME_PRECISION
col.Charset, // CHARACTER_SET_NAME
col.Collate, // COLLATION_NAME
columnType, // COLUMN_TYPE
columnDesc.Key, // COLUMN_KEY
columnDesc.Extra, // EXTRA
"select,insert,update,references", // PRIVILEGES
columnDesc.Comment, // COLUMN_COMMENT
col.GeneratedExprString, // GENERATION_EXPRESSION
charMaxLen, // CHARACTER_MAXIMUM_LENGTH
charOctLen, // CHARACTER_OCTET_LENGTH
numericPrecision, // NUMERIC_PRECISION
numericScale, // NUMERIC_SCALE
datetimePrecision, // DATETIME_PRECISION
col.Charset, // CHARACTER_SET_NAME
col.Collate, // COLLATION_NAME
columnType, // COLUMN_TYPE
columnDesc.Key, // COLUMN_KEY
columnDesc.Extra, // EXTRA
"select,insert,update,references", // PRIVILEGES
columnDesc.Comment, // COLUMN_COMMENT
col.GeneratedExprString, // GENERATION_EXPRESSION
)
// In mysql, 'character_set_name' and 'collation_name' are setted to null when column type is non-varchar or non-blob in information_schema.
if col.Tp != mysql.TypeVarchar && col.Tp != mysql.TypeBlob {
Expand Down Expand Up @@ -1180,7 +1193,7 @@ func (s schemasSorter) Less(i, j int) bool {
}

func (it *infoschemaTable) getRows(ctx sessionctx.Context, cols []*table.Column) (fullRows [][]types.Datum, err error) {
is := it.handle.Get()
is := ctx.GetSessionVars().TxnCtx.InfoSchema.(InfoSchema)
dbs := is.AllSchemas()
sort.Sort(schemasSorter(dbs))
switch it.meta.Name.O {
Expand Down
Loading

0 comments on commit 1ea1dc1

Please sign in to comment.