From fc739762c3c8411e929bc70ee697198c56ce9c77 Mon Sep 17 00:00:00 2001 From: HuaiyuXu <391585975@qq.com> Date: Fri, 30 Aug 2019 13:29:22 +0800 Subject: [PATCH 01/21] executor: refine the usage of expression blacklist (#11940) --- cmd/explaintest/r/black_list.result | 21 ++ cmd/explaintest/t/black_list.test | 16 +- executor/reload_expr_pushdown_blacklist.go | 269 ++++++++++++++++++++- 3 files changed, 304 insertions(+), 2 deletions(-) diff --git a/cmd/explaintest/r/black_list.result b/cmd/explaintest/r/black_list.result index 465490c35bfdb..e85d21339a3bf 100644 --- a/cmd/explaintest/r/black_list.result +++ b/cmd/explaintest/r/black_list.result @@ -17,6 +17,22 @@ Selection_5 8000.00 root lt(test.t.a, 1) delete from mysql.opt_rule_blacklist where name='predicate_push_down'; admin reload opt_rule_blacklist; +explain select * from t where a < 1; +id count task operator info +TableReader_7 3323.33 root data:Selection_6 +└─Selection_6 3323.33 cop lt(test.t.a, 1) + └─TableScan_5 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo +insert into mysql.expr_pushdown_blacklist values('<'); +admin reload expr_pushdown_blacklist; + +explain select * from t where a < 1; +id count task operator info +Selection_5 8000.00 root lt(test.t.a, 1) +└─TableReader_7 10000.00 root data:TableScan_6 + └─TableScan_6 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo +delete from mysql.expr_pushdown_blacklist where name='<'; +admin reload expr_pushdown_blacklist; + explain select * from t where a < 1; id count task operator info TableReader_7 3323.33 root data:Selection_6 @@ -33,3 +49,8 @@ Selection_5 8000.00 root lt(test.t.a, 1) delete from mysql.expr_pushdown_blacklist where name='lt'; admin reload expr_pushdown_blacklist; +explain select * from t where a < 1; +id count task operator info +TableReader_7 3323.33 root data:Selection_6 +└─Selection_6 3323.33 cop lt(test.t.a, 1) + └─TableScan_5 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo diff --git a/cmd/explaintest/t/black_list.test b/cmd/explaintest/t/black_list.test index 27a41226295cf..6e93cb4943046 100644 --- a/cmd/explaintest/t/black_list.test +++ b/cmd/explaintest/t/black_list.test @@ -16,6 +16,18 @@ admin reload opt_rule_blacklist; explain select * from t where a < 1; +insert into mysql.expr_pushdown_blacklist values('<'); + +admin reload expr_pushdown_blacklist; + +explain select * from t where a < 1; + +delete from mysql.expr_pushdown_blacklist where name='<'; + +admin reload expr_pushdown_blacklist; + +explain select * from t where a < 1; + insert into mysql.expr_pushdown_blacklist values('lt'); admin reload expr_pushdown_blacklist; @@ -24,4 +36,6 @@ explain select * from t where a < 1; delete from mysql.expr_pushdown_blacklist where name='lt'; -admin reload expr_pushdown_blacklist; \ No newline at end of file +admin reload expr_pushdown_blacklist; + +explain select * from t where a < 1; diff --git a/executor/reload_expr_pushdown_blacklist.go b/executor/reload_expr_pushdown_blacklist.go index 4bba2791d2047..d6892de79b80d 100644 --- a/executor/reload_expr_pushdown_blacklist.go +++ b/executor/reload_expr_pushdown_blacklist.go @@ -15,7 +15,9 @@ package executor import ( "context" + "strings" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/chunk" @@ -41,9 +43,274 @@ func LoadExprPushdownBlacklist(ctx sessionctx.Context) (err error) { } newBlacklist := make(map[string]struct{}) for _, row := range rows { - name := row.GetString(0) + name := strings.ToLower(row.GetString(0)) + if alias, ok := funcName2Alias[name]; ok { + name = alias + } newBlacklist[name] = struct{}{} } expression.DefaultExprPushdownBlacklist.Store(newBlacklist) return nil } + +// funcName2Alias indicates map of the origin function name to the name used in TiDB. +var funcName2Alias = map[string]string{ + "and": ast.LogicAnd, + "cast": ast.Cast, + "<<": ast.LeftShift, + ">>": ast.RightShift, + "or": ast.LogicOr, + ">=": ast.GE, + "<=": ast.LE, + "=": ast.EQ, + "!=": ast.NE, + "<>": ast.NE, + "<": ast.LT, + ">": ast.GT, + "+": ast.Plus, + "-": ast.Minus, + "&&": ast.And, + "||": ast.Or, + "%": ast.Mod, + "xor_bit": ast.Xor, + "/": ast.Div, + "*": ast.Mul, + "!": ast.UnaryNot, + "~": ast.BitNeg, + "div": ast.IntDiv, + "xor_logic": ast.LogicXor, // Avoid name conflict with "xor_bit"., + "<=>": ast.NullEQ, + "+_unary": ast.UnaryPlus, // Avoid name conflict with `plus`., + "-_unary": ast.UnaryMinus, + "in": ast.In, + "like": ast.Like, + "case": ast.Case, + "regexp": ast.Regexp, + "is null": ast.IsNull, + "is true": ast.IsTruth, + "is false": ast.IsFalsity, + "values": ast.Values, + "bit_count": ast.BitCount, + "coalesce": ast.Coalesce, + "greatest": ast.Greatest, + "least": ast.Least, + "interval": ast.Interval, + "abs": ast.Abs, + "acos": ast.Acos, + "asin": ast.Asin, + "atan": ast.Atan, + "atan2": ast.Atan2, + "ceil": ast.Ceil, + "ceiling": ast.Ceiling, + "conv": ast.Conv, + "cos": ast.Cos, + "cot": ast.Cot, + "crc32": ast.CRC32, + "degrees": ast.Degrees, + "exp": ast.Exp, + "floor": ast.Floor, + "ln": ast.Ln, + "log": ast.Log, + "log2": ast.Log2, + "log10": ast.Log10, + "pi": ast.PI, + "pow": ast.Pow, + "power": ast.Power, + "radians": ast.Radians, + "rand": ast.Rand, + "round": ast.Round, + "sign": ast.Sign, + "sin": ast.Sin, + "sqrt": ast.Sqrt, + "tan": ast.Tan, + "truncate": ast.Truncate, + "adddate": ast.AddDate, + "addtime": ast.AddTime, + "convert_tz": ast.ConvertTz, + "curdate": ast.Curdate, + "current_date": ast.CurrentDate, + "current_time": ast.CurrentTime, + "current_timestamp": ast.CurrentTimestamp, + "curtime": ast.Curtime, + "date": ast.Date, + "date_add": ast.DateAdd, + "date_format": ast.DateFormat, + "date_sub": ast.DateSub, + "datediff": ast.DateDiff, + "day": ast.Day, + "dayname": ast.DayName, + "dayofmonth": ast.DayOfMonth, + "dayofweek": ast.DayOfWeek, + "dayofyear": ast.DayOfYear, + "extract": ast.Extract, + "from_days": ast.FromDays, + "from_unixtime": ast.FromUnixTime, + "get_format": ast.GetFormat, + "hour": ast.Hour, + "localtime": ast.LocalTime, + "localtimestamp": ast.LocalTimestamp, + "makedate": ast.MakeDate, + "maketime": ast.MakeTime, + "microsecond": ast.MicroSecond, + "minute": ast.Minute, + "month": ast.Month, + "monthname": ast.MonthName, + "now": ast.Now, + "period_add": ast.PeriodAdd, + "period_diff": ast.PeriodDiff, + "quarter": ast.Quarter, + "sec_to_time": ast.SecToTime, + "second": ast.Second, + "str_to_date": ast.StrToDate, + "subdate": ast.SubDate, + "subtime": ast.SubTime, + "sysdate": ast.Sysdate, + "time": ast.Time, + "time_format": ast.TimeFormat, + "time_to_sec": ast.TimeToSec, + "timediff": ast.TimeDiff, + "timestamp": ast.Timestamp, + "timestampadd": ast.TimestampAdd, + "timestampdiff": ast.TimestampDiff, + "to_days": ast.ToDays, + "to_seconds": ast.ToSeconds, + "unix_timestamp": ast.UnixTimestamp, + "utc_date": ast.UTCDate, + "utc_time": ast.UTCTime, + "utc_timestamp": ast.UTCTimestamp, + "week": ast.Week, + "weekday": ast.Weekday, + "weekofyear": ast.WeekOfYear, + "year": ast.Year, + "yearweek": ast.YearWeek, + "last_day": ast.LastDay, + "ascii": ast.ASCII, + "bin": ast.Bin, + "concat": ast.Concat, + "concat_ws": ast.ConcatWS, + "convert": ast.Convert, + "elt": ast.Elt, + "export_set": ast.ExportSet, + "field": ast.Field, + "format": ast.Format, + "from_base64": ast.FromBase64, + "insert_func": ast.InsertFunc, + "instr": ast.Instr, + "lcase": ast.Lcase, + "left": ast.Left, + "length": ast.Length, + "load_file": ast.LoadFile, + "locate": ast.Locate, + "lower": ast.Lower, + "lpad": ast.Lpad, + "ltrim": ast.LTrim, + "make_set": ast.MakeSet, + "mid": ast.Mid, + "oct": ast.Oct, + "octet_length": ast.OctetLength, + "ord": ast.Ord, + "position": ast.Position, + "quote": ast.Quote, + "repeat": ast.Repeat, + "replace": ast.Replace, + "reverse": ast.Reverse, + "right": ast.Right, + "rtrim": ast.RTrim, + "space": ast.Space, + "strcmp": ast.Strcmp, + "substring": ast.Substring, + "substr": ast.Substr, + "substring_index": ast.SubstringIndex, + "to_base64": ast.ToBase64, + "trim": ast.Trim, + "upper": ast.Upper, + "ucase": ast.Ucase, + "hex": ast.Hex, + "unhex": ast.Unhex, + "rpad": ast.Rpad, + "bit_length": ast.BitLength, + "char_func": ast.CharFunc, + "char_length": ast.CharLength, + "character_length": ast.CharacterLength, + "find_in_set": ast.FindInSet, + "benchmark": ast.Benchmark, + "charset": ast.Charset, + "coercibility": ast.Coercibility, + "collation": ast.Collation, + "connection_id": ast.ConnectionID, + "current_user": ast.CurrentUser, + "current_role": ast.CurrentRole, + "database": ast.Database, + "found_rows": ast.FoundRows, + "last_insert_id": ast.LastInsertId, + "row_count": ast.RowCount, + "schema": ast.Schema, + "session_user": ast.SessionUser, + "system_user": ast.SystemUser, + "user": ast.User, + "if": ast.If, + "ifnull": ast.Ifnull, + "nullif": ast.Nullif, + "any_value": ast.AnyValue, + "default_func": ast.DefaultFunc, + "inet_aton": ast.InetAton, + "inet_ntoa": ast.InetNtoa, + "inet6_aton": ast.Inet6Aton, + "inet6_ntoa": ast.Inet6Ntoa, + "is_free_lock": ast.IsFreeLock, + "is_ipv4": ast.IsIPv4, + "is_ipv4_compat": ast.IsIPv4Compat, + "is_ipv4_mapped": ast.IsIPv4Mapped, + "is_ipv6": ast.IsIPv6, + "is_used_lock": ast.IsUsedLock, + "master_pos_wait": ast.MasterPosWait, + "name_const": ast.NameConst, + "release_all_locks": ast.ReleaseAllLocks, + "sleep": ast.Sleep, + "uuid": ast.UUID, + "uuid_short": ast.UUIDShort, + "get_lock": ast.GetLock, + "release_lock": ast.ReleaseLock, + "aes_decrypt": ast.AesDecrypt, + "aes_encrypt": ast.AesEncrypt, + "compress": ast.Compress, + "decode": ast.Decode, + "des_decrypt": ast.DesDecrypt, + "des_encrypt": ast.DesEncrypt, + "encode": ast.Encode, + "encrypt": ast.Encrypt, + "md5": ast.MD5, + "old_password": ast.OldPassword, + "password_func": ast.PasswordFunc, + "random_bytes": ast.RandomBytes, + "sha1": ast.SHA1, + "sha": ast.SHA, + "sha2": ast.SHA2, + "uncompress": ast.Uncompress, + "uncompressed_length": ast.UncompressedLength, + "validate_password_strength": ast.ValidatePasswordStrength, + "json_type": ast.JSONType, + "json_extract": ast.JSONExtract, + "json_unquote": ast.JSONUnquote, + "json_array": ast.JSONArray, + "json_object": ast.JSONObject, + "json_merge": ast.JSONMerge, + "json_set": ast.JSONSet, + "json_insert": ast.JSONInsert, + "json_replace": ast.JSONReplace, + "json_remove": ast.JSONRemove, + "json_contains": ast.JSONContains, + "json_contains_path": ast.JSONContainsPath, + "json_valid": ast.JSONValid, + "json_array_append": ast.JSONArrayAppend, + "json_array_insert": ast.JSONArrayInsert, + "json_merge_patch": ast.JSONMergePatch, + "json_merge_preserve": ast.JSONMergePreserve, + "json_pretty": ast.JSONPretty, + "json_quote": ast.JSONQuote, + "json_search": ast.JSONSearch, + "json_storage_size": ast.JSONStorageSize, + "json_depth": ast.JSONDepth, + "json_keys": ast.JSONKeys, + "json_length": ast.JSONLength, +} From c07fcc4a343414b4260cc706249383c01276782c Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Fri, 30 Aug 2019 13:41:05 +0800 Subject: [PATCH 02/21] executor: make the projection executor support vectorized expression evaluation (#11917) --- expression/bench_test.go | 121 +++++++++++++++++++++++++++++++ expression/builtin.go | 1 + expression/chunk_executor.go | 88 +++++++++++++++++++++- expression/evaluator.go | 7 ++ session/session.go | 1 + sessionctx/variable/session.go | 5 ++ sessionctx/variable/sysvar.go | 1 + sessionctx/variable/tidb_vars.go | 4 + sessionctx/variable/varsutil.go | 2 +- util/chunk/chunk.go | 10 +++ 10 files changed, 237 insertions(+), 3 deletions(-) diff --git a/expression/bench_test.go b/expression/bench_test.go index 79a481c1e6c8c..de674bf1399ce 100644 --- a/expression/bench_test.go +++ b/expression/bench_test.go @@ -17,6 +17,9 @@ package expression import ( "fmt" + "math/rand" + "reflect" + "strings" "testing" "time" @@ -189,3 +192,121 @@ func BenchmarkScalarFunctionClone(b *testing.B) { } b.ReportAllocs() } + +type vecExprBenchCase struct { + builtinFuncName string + retEvalType types.EvalType + childrenTypes []types.EvalType +} + +var vecExprBenchCases = []vecExprBenchCase{ + {ast.Cast, types.ETInt, []types.EvalType{types.ETInt}}, +} + +func fillColumn(eType types.EvalType, chk *chunk.Chunk, colIdx int) { + nullRatio := 0.2 + batchSize := 1024 + switch eType { + case types.ETInt: + for i := 0; i < batchSize; i++ { + if rand.Float64() < nullRatio { + chk.AppendNull(colIdx) + } else { + if rand.Float64() < 0.5 { + chk.AppendInt64(colIdx, -rand.Int63()) + } else { + chk.AppendInt64(colIdx, rand.Int63()) + } + } + } + default: + // TODO: support all EvalTypes later. + panic(fmt.Sprintf("EvalType=%v is not supported.", eType)) + } +} + +func eType2FieldType(eType types.EvalType) *types.FieldType { + switch eType { + case types.ETInt: + return types.NewFieldType(mysql.TypeLonglong) + } + // TODO: support all EvalTypes later. + panic(fmt.Sprintf("EvalType=%v is not supported.", eType)) +} + +func genVecExprBenchCase(ctx sessionctx.Context, testCase vecExprBenchCase) (Expression, *chunk.Chunk, *chunk.Chunk) { + fts := make([]*types.FieldType, len(testCase.childrenTypes)) + for i, eType := range testCase.childrenTypes { + fts[i] = eType2FieldType(eType) + } + cols := make([]Expression, len(testCase.childrenTypes)) + input := chunk.New(fts, 1024, 1024) + for i, eType := range testCase.childrenTypes { + fillColumn(eType, input, i) + cols[i] = &Column{Index: i, RetType: fts[i]} + } + + expr, err := NewFunction(ctx, testCase.builtinFuncName, eType2FieldType(testCase.retEvalType), cols...) + if err != nil { + panic(err) + } + + output := chunk.New([]*types.FieldType{eType2FieldType(testCase.retEvalType)}, 1024, 1024) + return expr, input, output +} + +func TestVectorizedExpression(t *testing.T) { + ctx := mock.NewContext() + for _, testCase := range vecExprBenchCases { + expr, input, output := genVecExprBenchCase(ctx, testCase) + output2 := output.CopyConstruct() + if err := evalOneVec(ctx, expr, input, output, 0); err != nil { + t.Fatal(err) + } + + it := chunk.NewIterator4Chunk(input) + if err := evalOneColumn(ctx, expr, it, output2, 0); err != nil { + t.Fatal(err) + } + + switch testCase.retEvalType { + case types.ETInt: + if !reflect.DeepEqual(output.Column(0).Int64s(), output2.Column(0).Int64s()) { + t.Fatal(fmt.Sprintf("error testCase %v", testCase)) + } + default: + t.Fatal(fmt.Sprintf("evalType=%v is not supported", testCase.retEvalType)) + } + } +} + +func BenchmarkVectorizedExpression(b *testing.B) { + ctx := mock.NewContext() + for _, testCase := range vecExprBenchCases { + expr, input, output := genVecExprBenchCase(ctx, testCase) + exprName := expr.String() + if sf, ok := expr.(*ScalarFunction); ok { + exprName = fmt.Sprintf("%v", reflect.TypeOf(sf.Function)) + tmp := strings.Split(exprName, ".") + exprName = tmp[len(tmp)-1] + } + + b.Run(exprName+"-VecExpr", func(b *testing.B) { + b.ResetTimer() + for i := 0; i < b.N; i++ { + if err := evalOneVec(ctx, expr, input, output, 0); err != nil { + b.Fatal(err) + } + } + }) + b.Run(exprName+"-NonVecExpr", func(b *testing.B) { + b.ResetTimer() + for i := 0; i < b.N; i++ { + it := chunk.NewIterator4Chunk(input) + if err := evalOneColumn(ctx, expr, it, output, 0); err != nil { + b.Fatal(err) + } + } + }) + } +} diff --git a/expression/builtin.go b/expression/builtin.go index 86b14ea6d2496..66d3ef3473eff 100644 --- a/expression/builtin.go +++ b/expression/builtin.go @@ -298,6 +298,7 @@ func (b *baseBuiltinFunc) cloneFrom(from *baseBuiltinFunc) { b.tp = from.tp b.pbCode = from.pbCode b.columnBufferAllocator = newLocalSliceBuffer(len(b.args)) + b.childrenVectorizedOnce = new(sync.Once) } func (b *baseBuiltinFunc) Clone() builtinFunc { diff --git a/expression/chunk_executor.go b/expression/chunk_executor.go index 6ee5a722ccc78..7294c9d85dd95 100644 --- a/expression/chunk_executor.go +++ b/expression/chunk_executor.go @@ -14,13 +14,12 @@ package expression import ( - "strconv" - "github.com/pingcap/parser/ast" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "strconv" ) // Vectorizable checks whether a list of expressions can employ vectorized execution. @@ -64,6 +63,91 @@ func VectorizedExecute(ctx sessionctx.Context, exprs []Expression, iterator *chu return nil } +func evalOneVec(ctx sessionctx.Context, expr Expression, input *chunk.Chunk, output *chunk.Chunk, colIdx int) error { + ft := expr.GetType() + result := output.Column(colIdx) + switch ft.EvalType() { + case types.ETInt: + if err := expr.VecEvalInt(ctx, input, result); err != nil { + return err + } + if ft.Tp == mysql.TypeBit { + i64s := result.Int64s() + buf := chunk.NewColumn(ft, input.NumRows()) + buf.ReserveBytes(input.NumRows()) + uintBuf := make([]byte, 8) + for i := range i64s { + if result.IsNull(i) { + buf.AppendNull() + } else { + buf.AppendBytes(strconv.AppendUint(uintBuf[:0], uint64(i64s[i]), 10)) + } + } + // TODO: recycle all old Columns returned here. + output.SetCol(colIdx, buf) + } else if mysql.HasUnsignedFlag(ft.Flag) { + // the underlying memory formats of int64 and uint64 are the same in Golang, + // so we can do a no-op here. + } + case types.ETReal: + if err := expr.VecEvalReal(ctx, input, result); err != nil { + return err + } + if ft.Tp == mysql.TypeFloat { + f64s := result.Float64s() + buf := chunk.NewColumn(ft, input.NumRows()) + buf.ResizeFloat32(input.NumRows()) + f32s := buf.Float32s() + for i := range f64s { + if result.IsNull(i) { + buf.SetNull(i, true) + } else { + f32s[i] = float32(f64s[i]) + } + } + output.SetCol(colIdx, buf) + } + case types.ETDecimal: + return expr.VecEvalDecimal(ctx, input, result) + case types.ETDatetime, types.ETTimestamp: + return expr.VecEvalTime(ctx, input, result) + case types.ETDuration: + return expr.VecEvalDuration(ctx, input, result) + case types.ETJson: + return expr.VecEvalJSON(ctx, input, result) + case types.ETString: + if err := expr.VecEvalString(ctx, input, result); err != nil { + return err + } + if ft.Tp == mysql.TypeEnum { + n := input.NumRows() + buf := chunk.NewColumn(ft, n) + buf.ReserveEnum(n) + for i := 0; i < n; i++ { + if result.IsNull(i) { + buf.AppendNull() + } else { + buf.AppendEnum(types.Enum{Value: 0, Name: result.GetString(i)}) + } + } + output.SetCol(colIdx, buf) + } else if ft.Tp == mysql.TypeSet { + n := input.NumRows() + buf := chunk.NewColumn(ft, n) + buf.ReserveSet(n) + for i := 0; i < n; i++ { + if result.IsNull(i) { + buf.AppendNull() + } else { + buf.AppendSet(types.Set{Value: 0, Name: result.GetString(i)}) + } + } + output.SetCol(colIdx, buf) + } + } + return nil +} + func evalOneColumn(ctx sessionctx.Context, expr Expression, iterator *chunk.Iterator4Chunk, output *chunk.Chunk, colID int) (err error) { switch fieldType, evalType := expr.GetType(), expr.GetType().EvalType(); evalType { case types.ETInt: diff --git a/expression/evaluator.go b/expression/evaluator.go index c0e401d69ffbb..bc2a5cb0924a2 100644 --- a/expression/evaluator.go +++ b/expression/evaluator.go @@ -47,6 +47,13 @@ func (e *defaultEvaluator) run(ctx sessionctx.Context, input, output *chunk.Chun iter := chunk.NewIterator4Chunk(input) if e.vectorizable { for i := range e.outputIdxes { + if ctx.GetSessionVars().EnableVectorizedExpression && e.exprs[i].Vectorized() { + if err := evalOneVec(ctx, e.exprs[i], input, output, e.outputIdxes[i]); err != nil { + return err + } + continue + } + err := evalOneColumn(ctx, e.exprs[i], iter, output, e.outputIdxes[i]) if err != nil { return err diff --git a/session/session.go b/session/session.go index 3086ed884b8ae..a801298f9a0c8 100644 --- a/session/session.go +++ b/session/session.go @@ -1703,6 +1703,7 @@ var builtinGlobalVariable = []string{ variable.TiDBRetryLimit, variable.TiDBDisableTxnAutoRetry, variable.TiDBEnableWindowFunction, + variable.TiDBEnableVectorizedExpression, variable.TiDBEnableFastAnalyze, variable.TiDBExpensiveQueryTimeThreshold, variable.TiDBEnableNoopFuncs, diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 4b97fe6f3ed11..a76ad0a70a701 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -339,6 +339,9 @@ type SessionVars struct { // EnableWindowFunction enables the window function. EnableWindowFunction bool + // EnableVectorizedExpression enables the vectorized expression evaluation. + EnableVectorizedExpression bool + // EnableIndexMerge enables the generation of IndexMergePath. EnableIndexMerge bool @@ -820,6 +823,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { s.EnableRadixJoin = TiDBOptOn(val) case TiDBEnableWindowFunction: s.EnableWindowFunction = TiDBOptOn(val) + case TiDBEnableVectorizedExpression: + s.EnableVectorizedExpression = TiDBOptOn(val) case TiDBOptJoinReorderThreshold: s.TiDBOptJoinReorderThreshold = tidbOptPositiveInt32(val, DefTiDBOptJoinReorderThreshold) case TiDBCheckMb4ValueInUTF8: diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 9d8ade09e4748..73a89e1982f3f 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -681,6 +681,7 @@ var defaultSysVars = []*SysVar{ {ScopeSession, TiDBTxnMode, DefTiDBTxnMode}, {ScopeSession, TiDBOptimizerSelectivityLevel, strconv.Itoa(DefTiDBOptimizerSelectivityLevel)}, {ScopeGlobal | ScopeSession, TiDBEnableWindowFunction, BoolToIntStr(DefEnableWindowFunction)}, + {ScopeGlobal | ScopeSession, TiDBEnableVectorizedExpression, BoolToIntStr(DefEnableVectorizedExpression)}, {ScopeGlobal | ScopeSession, TiDBEnableFastAnalyze, BoolToIntStr(DefTiDBUseFastAnalyze)}, {ScopeGlobal | ScopeSession, TiDBSkipIsolationLevelCheck, BoolToIntStr(DefTiDBSkipIsolationLevelCheck)}, /* The following variable is defined as session scope but is actually server scope. */ diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index a4ae090f59a5e..7c492d9db47fa 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -271,6 +271,9 @@ const ( // tidb_enable_window_function is used to control whether to enable the window function. TiDBEnableWindowFunction = "tidb_enable_window_function" + // tidb_enable_vectorized_expression is used to control whether to enable the vectorized expression evaluation. + TiDBEnableVectorizedExpression = "tidb_enable_vectorized_expression" + // TIDBOptJoinReorderThreshold defines the threshold less than which // we'll choose a rather time consuming algorithm to calculate the join order. TiDBOptJoinReorderThreshold = "tidb_opt_join_reorder_threshold" @@ -344,6 +347,7 @@ const ( DefTiDBForcePriority = mysql.NoPriority DefTiDBUseRadixJoin = false DefEnableWindowFunction = true + DefEnableVectorizedExpression = false DefTiDBOptJoinReorderThreshold = 0 DefTiDBDDLSlowOprThreshold = 300 DefTiDBUseFastAnalyze = false diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index ae244d93face2..227ed276d4bae 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -379,7 +379,7 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, TiDBBatchInsert, TiDBDisableTxnAutoRetry, TiDBEnableStreaming, TiDBBatchDelete, TiDBBatchCommit, TiDBEnableCascadesPlanner, TiDBEnableWindowFunction, TiDBCheckMb4ValueInUTF8, TiDBLowResolutionTSO, TiDBEnableIndexMerge, TiDBEnableNoopFuncs, - TiDBScatterRegion, TiDBGeneralLog, TiDBConstraintCheckInPlace: + TiDBScatterRegion, TiDBGeneralLog, TiDBConstraintCheckInPlace, TiDBEnableVectorizedExpression: fallthrough case GeneralLog, AvoidTemporalUpgrade, BigTables, CheckProxyUsers, LogBin, CoreFile, EndMakersInJSON, SQLLogBin, OfflineMode, PseudoSlaveMode, LowPriorityUpdates, diff --git a/util/chunk/chunk.go b/util/chunk/chunk.go index ffc7cd55abfa4..16ab7851fb058 100644 --- a/util/chunk/chunk.go +++ b/util/chunk/chunk.go @@ -609,6 +609,16 @@ func (c *Chunk) Column(colIdx int) *Column { return c.columns[colIdx] } +// SetCol sets the colIdx Column to col and returns the old Column. +func (c *Chunk) SetCol(colIdx int, col *Column) *Column { + if col == c.columns[colIdx] { + return nil + } + old := c.columns[colIdx] + c.columns[colIdx] = col + return old +} + // Sel returns Sel of this Chunk. func (c *Chunk) Sel() []int { return c.sel From 25e134cd6ea8bf1b44f5fdaed6b3c3c9f7a2d761 Mon Sep 17 00:00:00 2001 From: Lonng Date: Fri, 30 Aug 2019 16:01:06 +0800 Subject: [PATCH 03/21] ddl: fix add unique index on partitioned table (by RANGE COLUMNS) failed (#11946) --- ddl/db_integration_test.go | 28 ++++++++++++++++++++++++++++ ddl/ddl_api.go | 2 +- ddl/partition.go | 28 +++++++++++++++++++++++----- 3 files changed, 52 insertions(+), 6 deletions(-) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 8e8124b558d21..45df0e8f4715d 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -1647,6 +1647,34 @@ func (s *testIntegrationSuite3) TestAlterAlgorithm(c *C) { s.tk.MustExec("alter table t default charset = utf8mb4, ALGORITHM=INSTANT") } +func (s *testIntegrationSuite3) TestAlterTableAddUniqueOnPartionRangeColumn(c *C) { + s.tk = testkit.NewTestKit(c, s.store) + s.tk.MustExec("use test") + s.tk.MustExec("drop table if exists t") + 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 COLUMNS( 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.tk.MustExec("insert into t values (4, 'xxx', 4)") + s.tk.MustExec("insert into t values (4, 'xxx', 9)") // Note the repeated 4 + s.tk.MustExec("insert into t values (17, 'xxx', 12)") + assertErrorCode(c, s.tk, "alter table t add unique index idx_a(a)", mysql.ErrDupEntry) + + s.tk.MustExec("delete from t where a = 4") + s.tk.MustExec("alter table t add unique index idx_a(a)") + s.tk.MustExec("alter table t add unique index idx_ac(a, c)") + assertErrorCode(c, s.tk, "alter table t add unique index idx_b(b)", mysql.ErrUniqueKeyNeedAllFieldsInPf) +} + func (s *testIntegrationSuite5) TestFulltextIndexIgnore(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 523f03e0a6163..e5c287439d060 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3251,7 +3251,7 @@ func (d *ddl) CreateIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast.Inde return errors.Trace(err) } if unique && tblInfo.GetPartitionInfo() != nil { - if err := checkPartitionKeysConstraint(ctx, tblInfo.GetPartitionInfo().Expr, idxColNames, tblInfo); err != nil { + if err := checkPartitionKeysConstraint(tblInfo.GetPartitionInfo(), idxColNames, tblInfo); err != nil { return err } } diff --git a/ddl/partition.go b/ddl/partition.go index c45669c64cfb0..1f094d997c258 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/opcode" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" @@ -620,11 +621,28 @@ func checkRangePartitioningKeysConstraints(sctx sessionctx.Context, s *ast.Creat return nil } -func checkPartitionKeysConstraint(sctx sessionctx.Context, partExpr string, idxColNames []*ast.IndexColName, tblInfo *model.TableInfo) error { - // Parse partitioning key, extract the column names in the partitioning key to slice. - partCols, err := extractPartitionColumns(partExpr, tblInfo) - if err != nil { - return err +func checkPartitionKeysConstraint(pi *model.PartitionInfo, idxColNames []*ast.IndexColName, tblInfo *model.TableInfo) error { + var ( + partCols []*model.ColumnInfo + err error + ) + // The expr will be an empty string if the partition is defined by: + // CREATE TABLE t (...) PARTITION BY RANGE COLUMNS(...) + if partExpr := pi.Expr; partExpr != "" { + // Parse partitioning key, extract the column names in the partitioning key to slice. + partCols, err = extractPartitionColumns(partExpr, tblInfo) + if err != nil { + return err + } + } else { + partCols = make([]*model.ColumnInfo, 0, len(pi.Columns)) + for _, col := range pi.Columns { + colInfo := getColumnInfoByName(tblInfo, col.L) + if colInfo == nil { + return infoschema.ErrColumnNotExists.GenWithStackByArgs(col, tblInfo.Name) + } + partCols = append(partCols, colInfo) + } } // Every unique key on the table must use every column in the table's partitioning expression. From dc1f8ff031357283541262ddaf0a4f1b4cdd8182 Mon Sep 17 00:00:00 2001 From: wjHuang Date: Fri, 30 Aug 2019 16:35:06 +0800 Subject: [PATCH 04/21] planner: don't prune sleep function in LogicalProjection (#11927) --- cmd/explaintest/r/select.result | 5 +++++ cmd/explaintest/t/select.test | 3 +++ planner/core/rule_column_pruning.go | 12 ++++++------ 3 files changed, 14 insertions(+), 6 deletions(-) diff --git a/cmd/explaintest/r/select.result b/cmd/explaintest/r/select.result index 589a6683b8d01..c80b6cb8438ae 100644 --- a/cmd/explaintest/r/select.result +++ b/cmd/explaintest/r/select.result @@ -432,6 +432,11 @@ Projection_7 10000.00 root 6_aux_0 │ └─TableScan_9 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo └─TableReader_12 10000.00 root data:TableScan_11 └─TableScan_11 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo +explain select 1 from (select sleep(1)) t; +id count task operator info +Projection_4 1.00 root 1 +└─Projection_5 1.00 root sleep(1) + └─TableDual_6 1.00 root rows:1 drop table if exists t; create table t(a int, b int); explain select a from t order by rand(); diff --git a/cmd/explaintest/t/select.test b/cmd/explaintest/t/select.test index 3f3fb13d71031..d4a12f750585e 100644 --- a/cmd/explaintest/t/select.test +++ b/cmd/explaintest/t/select.test @@ -204,6 +204,9 @@ drop table t; create table t(a int not null, b int); explain select a in (select a from t t2 where t2.b = t1.b) from t t1; +# test sleep in subquery +explain select 1 from (select sleep(1)) t; + # test order by rand() drop table if exists t; create table t(a int, b int); diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index c0b3bbda3bccd..38e25f0aeca34 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -53,17 +53,17 @@ func getUsedList(usedCols []*expression.Column, schema *expression.Schema) ([]bo return used, nil } -// exprHasSetVar checks if the expression has SetVar function. -func exprHasSetVar(expr expression.Expression) bool { +// exprHasSetVarOrSleep checks if the expression has SetVar function or Sleep function. +func exprHasSetVarOrSleep(expr expression.Expression) bool { scalaFunc, isScalaFunc := expr.(*expression.ScalarFunction) if !isScalaFunc { return false } - if scalaFunc.FuncName.L == ast.SetVar { + if scalaFunc.FuncName.L == ast.SetVar || scalaFunc.FuncName.L == ast.Sleep { return true } for _, arg := range scalaFunc.GetArgs() { - if exprHasSetVar(arg) { + if exprHasSetVarOrSleep(arg) { return true } } @@ -71,7 +71,7 @@ func exprHasSetVar(expr expression.Expression) bool { } // PruneColumns implements LogicalPlan interface. -// If any expression has SetVar functions, we do not prune it. +// If any expression has SetVar function or Sleep function, we do not prune it. func (p *LogicalProjection) PruneColumns(parentUsedCols []*expression.Column) error { child := p.children[0] used, err := getUsedList(parentUsedCols, p.schema) @@ -80,7 +80,7 @@ func (p *LogicalProjection) PruneColumns(parentUsedCols []*expression.Column) er } for i := len(used) - 1; i >= 0; i-- { - if !used[i] && !exprHasSetVar(p.Exprs[i]) { + if !used[i] && !exprHasSetVarOrSleep(p.Exprs[i]) { p.schema.Columns = append(p.schema.Columns[:i], p.schema.Columns[i+1:]...) p.Exprs = append(p.Exprs[:i], p.Exprs[i+1:]...) } From e415ced76e5eb2f5b4d1790d4ea9007bbb8c7d42 Mon Sep 17 00:00:00 2001 From: bb7133 Date: Fri, 30 Aug 2019 17:18:35 +0800 Subject: [PATCH 05/21] types/json: fix an over-quoted bug in `BinaryJSON.Unquote` function (#11934) --- expression/builtin_json_test.go | 4 +++- types/json/binary_functions.go | 20 +++++++++----------- types/json/binary_test.go | 1 + 3 files changed, 13 insertions(+), 12 deletions(-) diff --git a/expression/builtin_json_test.go b/expression/builtin_json_test.go index 6b44ffdb0097b..8da86f1ec8c73 100644 --- a/expression/builtin_json_test.go +++ b/expression/builtin_json_test.go @@ -95,8 +95,10 @@ func (s *testEvaluatorSuite) TestJSONUnquote(c *C) { {`{"a": "b"}`, `{"a": "b"}`}, {`"hello,\"quoted string\",world"`, `hello,"quoted string",world`}, {`"hello,\"宽字符\",world"`, `hello,"宽字符",world`}, - {`Invalid Json string\tis OK`, `Invalid Json string is OK`}, + {`Invalid Json string\tis OK`, `Invalid Json string\tis OK`}, {`"1\\u2232\\u22322"`, `1\u2232\u22322`}, + {`"[{\"x\":\"{\\\"y\\\":12}\"}]"`, `[{"x":"{\"y\":12}"}]`}, + {`[{\"x\":\"{\\\"y\\\":12}\"}]`, `[{\"x\":\"{\\\"y\\\":12}\"}]`}, } dtbl := tblToDtbl(tbl) for _, t := range dtbl { diff --git a/types/json/binary_functions.go b/types/json/binary_functions.go index 18dc0d2eed1b3..4a0cfd50d82ba 100644 --- a/types/json/binary_functions.go +++ b/types/json/binary_functions.go @@ -66,19 +66,17 @@ func (bj BinaryJSON) Unquote() (string, error) { switch bj.TypeCode { case TypeCodeString: tmp := string(hack.String(bj.GetString())) - s, err := unquoteString(tmp) - if err != nil { - return "", errors.Trace(err) + tlen := len(tmp) + if tlen < 2 { + return tmp, nil } - // Remove prefix and suffix '"'. - slen := len(s) - if slen > 1 { - head, tail := s[0], s[slen-1] - if head == '"' && tail == '"' { - return s[1 : slen-1], nil - } + head, tail := tmp[0], tmp[tlen-1] + if head == '"' && tail == '"' { + // Remove prefix and suffix '"' before unquoting + return unquoteString(tmp[1 : tlen-1]) } - return s, nil + // if value is not double quoted, do nothing + return tmp, nil default: return bj.String(), nil } diff --git a/types/json/binary_test.go b/types/json/binary_test.go index e31a65fb0e23c..dc4e997002e81 100644 --- a/types/json/binary_test.go +++ b/types/json/binary_test.go @@ -114,6 +114,7 @@ func (s *testJSONSuite) TestBinaryJSONUnquote(c *C) { }{ {j: `3`, unquoted: "3"}, {j: `"3"`, unquoted: "3"}, + {j: `"[{\"x\":\"{\\\"y\\\":12}\"}]"`, unquoted: `[{"x":"{\"y\":12}"}]`}, {j: `"hello, \"escaped quotes\" world"`, unquoted: "hello, \"escaped quotes\" world"}, {j: "\"\\u4f60\"", unquoted: "你"}, {j: `true`, unquoted: "true"}, From bb84d1bdbaa935d7d1f5670d523e70f692f7fd7a Mon Sep 17 00:00:00 2001 From: Maxwell Date: Fri, 30 Aug 2019 17:45:40 +0800 Subject: [PATCH 06/21] planner: allow insert `default` into generated columns (#11901) --- ddl/db_integration_test.go | 62 +++++++++++++++++++++++++++++++++++++ executor/insert_common.go | 7 ++--- planner/core/planbuilder.go | 24 ++++++++++---- 3 files changed, 83 insertions(+), 10 deletions(-) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 45df0e8f4715d..2d9581ffd7d21 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -1959,3 +1959,65 @@ func (s *testIntegrationSuite4) TestDropAutoIncrementIndex(c *C) { dropIndexSQL = "alter table t1 drop index a" assertErrorCode(c, tk, dropIndexSQL, mysql.ErrWrongAutoKey) } + +func (s *testIntegrationSuite3) TestInsertIntoGeneratedColumnWithDefaultExpr(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test") + tk.MustExec("use test") + + // insert into virtual / stored columns + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1 (a int, b int as (-a) virtual, c int as (-a) stored)") + tk.MustExec("insert into t1 values (1, default, default)") + tk.MustQuery("select * from t1").Check(testkit.Rows("1 -1 -1")) + tk.MustExec("delete from t1") + + // insert multiple rows + tk.MustExec("insert into t1(a,b) values (1, default), (2, default)") + tk.MustQuery("select * from t1").Check(testkit.Rows("1 -1 -1", "2 -2 -2")) + tk.MustExec("delete from t1") + + // insert into generated columns only + tk.MustExec("insert into t1(b) values (default)") + tk.MustQuery("select * from t1").Check(testkit.Rows(" ")) + tk.MustExec("delete from t1") + tk.MustExec("insert into t1(c) values (default)") + tk.MustQuery("select * from t1").Check(testkit.Rows(" ")) + tk.MustExec("delete from t1") + + // generated columns with index + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t2 like t1") + tk.MustExec("alter table t2 add index idx1(a)") + tk.MustExec("alter table t2 add index idx2(b)") + tk.MustExec("insert into t2 values (1, default, default)") + tk.MustQuery("select * from t2").Check(testkit.Rows("1 -1 -1")) + tk.MustExec("delete from t2") + tk.MustExec("alter table t2 drop index idx1") + tk.MustExec("alter table t2 drop index idx2") + tk.MustExec("insert into t2 values (1, default, default)") + tk.MustQuery("select * from t2").Check(testkit.Rows("1 -1 -1")) + + // generated columns in different position + tk.MustExec("drop table if exists t3") + tk.MustExec("create table t3 (gc1 int as (r+1), gc2 int as (r+1) stored, gc3 int as (gc2+1), gc4 int as (gc1+1) stored, r int)") + tk.MustExec("insert into t3 values (default, default, default, default, 1)") + tk.MustQuery("select * from t3").Check(testkit.Rows("2 2 3 3 1")) + + // generated columns in replace statement + tk.MustExec("create table t4 (a int key, b int, c int as (a+1), d int as (b+1) stored)") + tk.MustExec("insert into t4 values (1, 10, default, default)") + tk.MustQuery("select * from t4").Check(testkit.Rows("1 10 2 11")) + tk.MustExec("replace into t4 values (1, 20, default, default)") + tk.MustQuery("select * from t4").Check(testkit.Rows("1 20 2 21")) + + // generated columns with default function is not allowed + tk.MustExec("create table t5 (a int default 10, b int as (a+1))") + assertErrorCode(c, tk, "insert into t5 values (20, default(a))", mysql.ErrBadGeneratedColumn) + + tk.MustExec("drop table t1") + tk.MustExec("drop table t2") + tk.MustExec("drop table t3") + tk.MustExec("drop table t4") + tk.MustExec("drop table t5") +} diff --git a/executor/insert_common.go b/executor/insert_common.go index 2b34afcf8735e..7fba881845d35 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -116,9 +116,6 @@ func (e *InsertValues) initInsertColumns() error { for _, v := range e.Columns { columns = append(columns, v.Name.O) } - for _, v := range e.GenColumns { - columns = append(columns, v.Name.O) - } cols, err = table.FindCols(tableCols, columns, e.Table.Meta().PKIsHandle) if err != nil { return errors.Errorf("INSERT INTO %s: %s", e.Table.Meta().Name.O, err) @@ -128,6 +125,9 @@ func (e *InsertValues) initInsertColumns() error { cols = tableCols } for _, col := range cols { + if !col.IsGenerated() { + e.insertColumns = append(e.insertColumns, col) + } if col.Name.L == model.ExtraHandleName.L { if !e.ctx.GetSessionVars().AllowWriteRowID { return errors.Errorf("insert, update and replace statements for _tidb_rowid are not supported.") @@ -142,7 +142,6 @@ func (e *InsertValues) initInsertColumns() error { if err != nil { return err } - e.insertColumns = cols return nil } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index eaa70b419cd44..c1c4eb65e87dd 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1762,12 +1762,6 @@ func (b *PlanBuilder) buildValuesListOfInsert(ctx context.Context, insert *ast.I if len(insert.Lists[0]) != len(affectedValuesCols) { return ErrWrongValueCountOnRow.GenWithStackByArgs(1) } - // No generated column is allowed. - for _, col := range affectedValuesCols { - if col.IsGenerated() { - return ErrBadGeneratedColumn.GenWithStackByArgs(col.Name.O, insertPlan.Table.Meta().Name.O) - } - } } insertPlan.AllAssignmentsAreConstant = true @@ -1785,8 +1779,17 @@ func (b *PlanBuilder) buildValuesListOfInsert(ctx context.Context, insert *ast.I for j, valueItem := range valuesItem { var expr expression.Expression var err error + var generatedColumnWithDefaultExpr bool + col := affectedValuesCols[j] switch x := valueItem.(type) { case *ast.DefaultExpr: + if col.IsGenerated() { + if x.Name != nil { + return ErrBadGeneratedColumn.GenWithStackByArgs(col.Name.O, insertPlan.Table.Meta().Name.O) + } + generatedColumnWithDefaultExpr = true + break + } if x.Name != nil { expr, err = b.findDefaultValue(totalTableCols, x.Name) } else { @@ -1807,6 +1810,15 @@ func (b *PlanBuilder) buildValuesListOfInsert(ctx context.Context, insert *ast.I _, isConstant := expr.(*expression.Constant) insertPlan.AllAssignmentsAreConstant = isConstant } + // insert value into a generated column is not allowed + if col.IsGenerated() { + // but there is only one exception: + // it is allowed to insert the `default` value into a generated column + if generatedColumnWithDefaultExpr { + continue + } + return ErrBadGeneratedColumn.GenWithStackByArgs(col.Name.O, insertPlan.Table.Meta().Name.O) + } exprList = append(exprList, expr) } insertPlan.Lists = append(insertPlan.Lists, exprList) From bbb70be448d4087773cdff3fb91da68f2cc1fccc Mon Sep 17 00:00:00 2001 From: Lonng Date: Sun, 1 Sep 2019 09:08:12 +0800 Subject: [PATCH 07/21] executor: uncomment the BatchPointGet unit tests (#11962) Signed-off-by: Lonng --- executor/batch_point_get_test.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/executor/batch_point_get_test.go b/executor/batch_point_get_test.go index 1efdde70aac26..be0f10af2829c 100644 --- a/executor/batch_point_get_test.go +++ b/executor/batch_point_get_test.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/util/testkit" ) type testBatchPointGetSuite struct { @@ -56,7 +57,7 @@ func (s *testBatchPointGetSuite) TearDownSuite(c *C) { } func (s *testBatchPointGetSuite) TestBatchPointGetExec(c *C) { - /*tk := testkit.NewTestKit(c, s.store) + tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int primary key auto_increment not null, b int, c int, unique key idx_abc(a, b, c))") @@ -97,5 +98,5 @@ func (s *testBatchPointGetSuite) TestBatchPointGetExec(c *C) { "1", "2", "4", - ))*/ + )) } From 5352e7c65c3c200bef65d6897840ffa322b0650e Mon Sep 17 00:00:00 2001 From: Lonng Date: Mon, 2 Sep 2019 10:09:44 +0800 Subject: [PATCH 08/21] ddl: use the Options to reorganize the NewDDL paramaters (#11966) --- ddl/column_change_test.go | 6 ++- ddl/column_test.go | 24 ++++++++++-- ddl/ddl.go | 28 +++++++------- ddl/ddl_test.go | 7 ---- ddl/ddl_worker_test.go | 78 ++++++++++++++++++++++++++++++++------- ddl/fail_test.go | 6 ++- ddl/foreign_key_test.go | 6 ++- ddl/index_change_test.go | 6 ++- ddl/options.go | 78 +++++++++++++++++++++++++++++++++++++++ ddl/options_test.go | 60 ++++++++++++++++++++++++++++++ ddl/reorg_test.go | 18 +++++++-- ddl/schema_test.go | 24 ++++++++++-- ddl/stat_test.go | 6 ++- ddl/table_test.go | 6 ++- ddl/util/syncer_test.go | 14 ++++++- domain/domain.go | 10 ++++- domain/domain_test.go | 8 +++- owner/manager_test.go | 28 ++++++++++++-- 18 files changed, 354 insertions(+), 59 deletions(-) create mode 100644 ddl/options.go create mode 100644 ddl/options_test.go diff --git a/ddl/column_change_test.go b/ddl/column_change_test.go index 62eb19840cb24..bf45932628794 100644 --- a/ddl/column_change_test.go +++ b/ddl/column_change_test.go @@ -59,7 +59,11 @@ func (s *testColumnChangeSuite) TearDownSuite(c *C) { } func (s *testColumnChangeSuite) TestColumnChange(c *C) { - d := testNewDDL(context.Background(), nil, s.store, nil, nil, testLease) + d := newDDL( + context.Background(), + WithStore(s.store), + WithLease(testLease), + ) defer d.Stop() // create table t (c1 int, c2 int); tblInfo := testTableInfo(c, d, "t", 2) diff --git a/ddl/column_test.go b/ddl/column_test.go index b5a7dab8d7690..9bc32081ae402 100644 --- a/ddl/column_test.go +++ b/ddl/column_test.go @@ -45,7 +45,11 @@ type testColumnSuite struct { func (s *testColumnSuite) SetUpSuite(c *C) { s.store = testCreateStore(c, "test_column") - s.d = testNewDDL(context.Background(), nil, s.store, nil, nil, testLease) + s.d = newDDL( + context.Background(), + WithStore(s.store), + WithLease(testLease), + ) s.dbInfo = testSchemaInfo(c, s.d, "test_column") testCreateSchema(c, testNewContext(s.d), s.d, s.dbInfo) @@ -753,7 +757,11 @@ func (s *testColumnSuite) testGetColumn(t table.Table, name string, isExist bool } func (s *testColumnSuite) TestAddColumn(c *C) { - d := testNewDDL(context.Background(), nil, s.store, nil, nil, testLease) + d := newDDL( + context.Background(), + WithStore(s.store), + WithLease(testLease), + ) tblInfo := testTableInfo(c, d, "t", 3) ctx := testNewContext(d) @@ -842,7 +850,11 @@ func (s *testColumnSuite) TestAddColumn(c *C) { } func (s *testColumnSuite) TestDropColumn(c *C) { - d := testNewDDL(context.Background(), nil, s.store, nil, nil, testLease) + d := newDDL( + context.Background(), + WithStore(s.store), + WithLease(testLease), + ) tblInfo := testTableInfo(c, d, "t", 4) ctx := testNewContext(d) @@ -919,7 +931,11 @@ func (s *testColumnSuite) TestDropColumn(c *C) { } func (s *testColumnSuite) TestModifyColumn(c *C) { - d := testNewDDL(context.Background(), nil, s.store, nil, nil, testLease) + d := newDDL( + context.Background(), + WithStore(s.store), + WithLease(testLease), + ) defer d.Stop() tests := []struct { origin string diff --git a/ddl/ddl.go b/ddl/ddl.go index 87072f7bcd75c..066d05efd7d36 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -23,7 +23,6 @@ import ( "sync" "time" - "github.com/coreos/etcd/clientv3" "github.com/ngaut/pools" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -346,21 +345,22 @@ func asyncNotifyEvent(d *ddlCtx, e *util.Event) { } // NewDDL creates a new DDL. -func NewDDL(ctx context.Context, etcdCli *clientv3.Client, store kv.Storage, - infoHandle *infoschema.Handle, hook Callback, lease time.Duration, ctxPool *pools.ResourcePool) DDL { - return newDDL(ctx, etcdCli, store, infoHandle, hook, lease, ctxPool) +func NewDDL(ctx context.Context, options ...Option) DDL { + return newDDL(ctx, options...) } -func newDDL(ctx context.Context, etcdCli *clientv3.Client, store kv.Storage, - infoHandle *infoschema.Handle, hook Callback, lease time.Duration, ctxPool *pools.ResourcePool) *ddl { - if hook == nil { - hook = &BaseCallback{} +func newDDL(ctx context.Context, options ...Option) *ddl { + opt := &Options{ + Hook: &BaseCallback{}, + } + for _, o := range options { + o(opt) } id := uuid.NewV4().String() ctx, cancelFunc := context.WithCancel(ctx) var manager owner.Manager var syncer util.SchemaSyncer - if etcdCli == nil { + if etcdCli := opt.EtcdCli; etcdCli == nil { // The etcdCli is nil if the store is localstore which is only used for testing. // So we use mockOwnerManager and MockSchemaSyncer. manager = owner.NewMockManager(id, cancelFunc) @@ -372,21 +372,21 @@ func newDDL(ctx context.Context, etcdCli *clientv3.Client, store kv.Storage, ddlCtx := &ddlCtx{ uuid: id, - store: store, - lease: lease, + store: opt.Store, + lease: opt.Lease, ddlJobDoneCh: make(chan struct{}, 1), ownerManager: manager, schemaSyncer: syncer, binlogCli: binloginfo.GetPumpsClient(), - infoHandle: infoHandle, + infoHandle: opt.InfoHandle, } - ddlCtx.mu.hook = hook + ddlCtx.mu.hook = opt.Hook ddlCtx.mu.interceptor = &BaseInterceptor{} d := &ddl{ ddlCtx: ddlCtx, } - d.start(ctx, ctxPool) + d.start(ctx, opt.ResourcePool) variable.RegisterStatistics(d) metrics.DDLCounter.WithLabelValues(metrics.CreateDDLInstance).Inc() diff --git a/ddl/ddl_test.go b/ddl/ddl_test.go index 1a0d52296f85e..cfd4fae805d62 100644 --- a/ddl/ddl_test.go +++ b/ddl/ddl_test.go @@ -19,13 +19,11 @@ import ( "testing" "time" - "github.com/coreos/etcd/clientv3" . "github.com/pingcap/check" "github.com/pingcap/log" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/terror" - "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" @@ -116,11 +114,6 @@ func testNewContext(d *ddl) sessionctx.Context { return ctx } -func testNewDDL(ctx context.Context, etcdCli *clientv3.Client, store kv.Storage, - infoHandle *infoschema.Handle, hook Callback, lease time.Duration) *ddl { - return newDDL(ctx, etcdCli, store, infoHandle, hook, lease, nil) -} - func getSchemaVer(c *C, ctx sessionctx.Context) int64 { err := ctx.NewTxn(context.Background()) c.Assert(err, IsNil) diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index 7b28a292f4403..5da6c2aeb56a4 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -53,7 +53,11 @@ func (s *testDDLSuite) TestCheckOwner(c *C) { store := testCreateStore(c, "test_owner") defer store.Close() - d1 := testNewDDL(context.Background(), nil, store, nil, nil, testLease) + d1 := newDDL( + context.Background(), + WithStore(store), + WithLease(testLease), + ) defer d1.Stop() time.Sleep(testLease) testCheckOwner(c, d1, true) @@ -67,7 +71,11 @@ func (s *testDDLSuite) testRunWorker(c *C) { defer store.Close() RunWorker = false - d := testNewDDL(context.Background(), nil, store, nil, nil, testLease) + d := newDDL( + context.Background(), + WithStore(store), + WithLease(testLease), + ) testCheckOwner(c, d, false) defer d.Stop() @@ -76,7 +84,11 @@ func (s *testDDLSuite) testRunWorker(c *C) { c.Assert(worker, IsNil) // Make sure the DDL job can be done and exit that goroutine. RunWorker = true - d1 := testNewDDL(context.Background(), nil, store, nil, nil, testLease) + d1 := newDDL( + context.Background(), + WithStore(store), + WithLease(testLease), + ) testCheckOwner(c, d1, true) defer d1.Stop() worker = d1.generalWorker() @@ -87,7 +99,11 @@ func (s *testDDLSuite) TestSchemaError(c *C) { store := testCreateStore(c, "test_schema_error") defer store.Close() - d := testNewDDL(context.Background(), nil, store, nil, nil, testLease) + d := newDDL( + context.Background(), + WithStore(store), + WithLease(testLease), + ) defer d.Stop() ctx := testNewContext(d) @@ -98,7 +114,11 @@ func (s *testDDLSuite) TestTableError(c *C) { store := testCreateStore(c, "test_table_error") defer store.Close() - d := testNewDDL(context.Background(), nil, store, nil, nil, testLease) + d := newDDL( + context.Background(), + WithStore(store), + WithLease(testLease), + ) defer d.Stop() ctx := testNewContext(d) @@ -139,7 +159,11 @@ func (s *testDDLSuite) TestViewError(c *C) { store := testCreateStore(c, "test_view_error") defer store.Close() - d := testNewDDL(context.Background(), nil, store, nil, nil, testLease) + d := newDDL( + context.Background(), + WithStore(store), + WithLease(testLease), + ) defer d.Stop() ctx := testNewContext(d) dbInfo := testSchemaInfo(c, d, "test") @@ -162,7 +186,11 @@ func (s *testDDLSuite) TestViewError(c *C) { func (s *testDDLSuite) TestInvalidDDLJob(c *C) { store := testCreateStore(c, "test_invalid_ddl_job_type_error") defer store.Close() - d := testNewDDL(context.Background(), nil, store, nil, nil, testLease) + d := newDDL( + context.Background(), + WithStore(store), + WithLease(testLease), + ) defer d.Stop() ctx := testNewContext(d) @@ -181,7 +209,11 @@ func (s *testDDLSuite) TestForeignKeyError(c *C) { store := testCreateStore(c, "test_foreign_key_error") defer store.Close() - d := testNewDDL(context.Background(), nil, store, nil, nil, testLease) + d := newDDL( + context.Background(), + WithStore(store), + WithLease(testLease), + ) defer d.Stop() ctx := testNewContext(d) @@ -199,7 +231,11 @@ func (s *testDDLSuite) TestIndexError(c *C) { store := testCreateStore(c, "test_index_error") defer store.Close() - d := testNewDDL(context.Background(), nil, store, nil, nil, testLease) + d := newDDL( + context.Background(), + WithStore(store), + WithLease(testLease), + ) defer d.Stop() ctx := testNewContext(d) @@ -234,7 +270,11 @@ func (s *testDDLSuite) TestIndexError(c *C) { func (s *testDDLSuite) TestColumnError(c *C) { store := testCreateStore(c, "test_column_error") defer store.Close() - d := testNewDDL(context.Background(), nil, store, nil, nil, testLease) + d := newDDL( + context.Background(), + WithStore(store), + WithLease(testLease), + ) defer d.Stop() ctx := testNewContext(d) @@ -443,7 +483,11 @@ func (s *testDDLSuite) checkCancelDropColumn(c *C, d *ddl, schemaID int64, table func (s *testDDLSuite) TestCancelJob(c *C) { store := testCreateStore(c, "test_cancel_job") defer store.Close() - d := testNewDDL(context.Background(), nil, store, nil, nil, testLease) + d := newDDL( + context.Background(), + WithStore(store), + WithLease(testLease), + ) defer d.Stop() dbInfo := testSchemaInfo(c, d, "test_cancel_job") testCreateSchema(c, testNewContext(d), d, dbInfo) @@ -850,7 +894,11 @@ func (s *testDDLSuite) TestBuildJobDependence(c *C) { func (s *testDDLSuite) TestParallelDDL(c *C) { store := testCreateStore(c, "test_parallel_ddl") defer store.Close() - d := testNewDDL(context.Background(), nil, store, nil, nil, testLease) + d := newDDL( + context.Background(), + WithStore(store), + WithLease(testLease), + ) defer d.Stop() ctx := testNewContext(d) err := ctx.NewTxn(context.Background()) @@ -1040,7 +1088,11 @@ func (s *testDDLSuite) TestDDLPackageExecuteSQL(c *C) { store := testCreateStore(c, "test_run_sql") defer store.Close() - d := testNewDDL(context.Background(), nil, store, nil, nil, testLease) + d := newDDL( + context.Background(), + WithStore(store), + WithLease(testLease), + ) testCheckOwner(c, d, true) defer d.Stop() worker := d.generalWorker() diff --git a/ddl/fail_test.go b/ddl/fail_test.go index 1e8f6ef2df4bd..e55b6e664fe0c 100644 --- a/ddl/fail_test.go +++ b/ddl/fail_test.go @@ -24,7 +24,11 @@ import ( ) func (s *testColumnChangeSuite) TestFailBeforeDecodeArgs(c *C) { - d := testNewDDL(context.Background(), nil, s.store, nil, nil, testLease) + d := newDDL( + context.Background(), + WithStore(s.store), + WithLease(testLease), + ) defer d.Stop() // create table t_fail (c1 int, c2 int); tblInfo := testTableInfo(c, d, "t_fail", 2) diff --git a/ddl/foreign_key_test.go b/ddl/foreign_key_test.go index cc3ed65e3ca6a..8740e2a49be47 100644 --- a/ddl/foreign_key_test.go +++ b/ddl/foreign_key_test.go @@ -111,7 +111,11 @@ func getForeignKey(t table.Table, name string) *model.FKInfo { } func (s *testForeighKeySuite) TestForeignKey(c *C) { - d := testNewDDL(context.Background(), nil, s.store, nil, nil, testLease) + d := newDDL( + context.Background(), + WithStore(s.store), + WithLease(testLease), + ) defer d.Stop() s.d = d s.dbInfo = testSchemaInfo(c, d, "test_foreign") diff --git a/ddl/index_change_test.go b/ddl/index_change_test.go index 770b5a5bfba02..d80f3eee5ecec 100644 --- a/ddl/index_change_test.go +++ b/ddl/index_change_test.go @@ -52,7 +52,11 @@ func (s *testIndexChangeSuite) TearDownSuite(c *C) { } func (s *testIndexChangeSuite) TestIndexChange(c *C) { - d := testNewDDL(context.Background(), nil, s.store, nil, nil, testLease) + d := newDDL( + context.Background(), + WithStore(s.store), + WithLease(testLease), + ) defer d.Stop() // create table t (c1 int primary key, c2 int); tblInfo := testTableInfo(c, d, "t", 2) diff --git a/ddl/options.go b/ddl/options.go new file mode 100644 index 0000000000000..729c0faa4d0da --- /dev/null +++ b/ddl/options.go @@ -0,0 +1,78 @@ +// Copyright 2019 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://wwm.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 ( + "time" + + "github.com/coreos/etcd/clientv3" + "github.com/ngaut/pools" + "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/kv" +) + +// Option represents an option to initialize the DDL module +type Option func(*Options) + +// Options represents all the options of the DDL module needs +type Options struct { + EtcdCli *clientv3.Client + Store kv.Storage + InfoHandle *infoschema.Handle + Hook Callback + Lease time.Duration + ResourcePool *pools.ResourcePool +} + +// WithEtcdClient specifies the `clientv3.Client` of DDL used to request the etcd service +func WithEtcdClient(client *clientv3.Client) Option { + return func(options *Options) { + options.EtcdCli = client + } +} + +// WithStore specifies the `kv.Storage` of DDL used to request the KV service +func WithStore(store kv.Storage) Option { + return func(options *Options) { + options.Store = store + } +} + +// WithInfoHandle specifies the `infoschema.Handle` +func WithInfoHandle(ih *infoschema.Handle) Option { + return func(options *Options) { + options.InfoHandle = ih + } +} + +// WithHook specifies the `Callback` of DDL used to notify the outer module when events are triggered +func WithHook(callback Callback) Option { + return func(options *Options) { + options.Hook = callback + } +} + +// WithLease specifies the schema lease duration +func WithLease(lease time.Duration) Option { + return func(options *Options) { + options.Lease = lease + } +} + +// WithResourcePool specifies the `pools.ResourcePool` of DDL used +func WithResourcePool(pools *pools.ResourcePool) Option { + return func(options *Options) { + options.ResourcePool = pools + } +} diff --git a/ddl/options_test.go b/ddl/options_test.go new file mode 100644 index 0000000000000..9ea6e6523b709 --- /dev/null +++ b/ddl/options_test.go @@ -0,0 +1,60 @@ +// Copyright 2019 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 ( + "time" + + "github.com/coreos/etcd/clientv3" + "github.com/ngaut/pools" + . "github.com/pingcap/check" + "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/util/mock" +) + +type ddlOptionsSuite struct{} + +var _ = Suite(&ddlOptionsSuite{}) + +func (s *ddlOptionsSuite) TestOptions(c *C) { + client, err := clientv3.NewFromURL("test") + c.Assert(err, IsNil) + callback := &ddl.BaseCallback{} + lease := time.Second * 3 + store := &mock.Store{} + infoHandle := infoschema.NewHandle(store) + pools := &pools.ResourcePool{} + + options := []ddl.Option{ + ddl.WithEtcdClient(client), + ddl.WithHook(callback), + ddl.WithLease(lease), + ddl.WithStore(store), + ddl.WithInfoHandle(infoHandle), + ddl.WithResourcePool(pools), + } + + opt := &ddl.Options{} + for _, o := range options { + o(opt) + } + + c.Assert(opt.EtcdCli, Equals, client) + c.Assert(opt.Hook, Equals, callback) + c.Assert(opt.Lease, Equals, lease) + c.Assert(opt.Store, Equals, store) + c.Assert(opt.InfoHandle, Equals, infoHandle) + c.Assert(opt.ResourcePool, Equals, pools) +} diff --git a/ddl/reorg_test.go b/ddl/reorg_test.go index 6ceff6b7687ac..3eb5e12d9906f 100644 --- a/ddl/reorg_test.go +++ b/ddl/reorg_test.go @@ -36,7 +36,11 @@ func (s *testDDLSuite) TestReorg(c *C) { store := testCreateStore(c, "test_reorg") defer store.Close() - d := testNewDDL(context.Background(), nil, store, nil, nil, testLease) + d := newDDL( + context.Background(), + WithStore(store), + WithLease(testLease), + ) defer d.Stop() time.Sleep(testLease) @@ -159,14 +163,22 @@ func (s *testDDLSuite) TestReorgOwner(c *C) { store := testCreateStore(c, "test_reorg_owner") defer store.Close() - d1 := testNewDDL(context.Background(), nil, store, nil, nil, testLease) + d1 := newDDL( + context.Background(), + WithStore(store), + WithLease(testLease), + ) defer d1.Stop() ctx := testNewContext(d1) testCheckOwner(c, d1, true) - d2 := testNewDDL(context.Background(), nil, store, nil, nil, testLease) + d2 := newDDL( + context.Background(), + WithStore(store), + WithLease(testLease), + ) defer d2.Stop() dbInfo := testSchemaInfo(c, d1, "test") diff --git a/ddl/schema_test.go b/ddl/schema_test.go index bc7e06ace9317..b9c2485a08189 100644 --- a/ddl/schema_test.go +++ b/ddl/schema_test.go @@ -125,7 +125,11 @@ func testCheckSchemaState(c *C, d *ddl, dbInfo *model.DBInfo, state model.Schema func (s *testSchemaSuite) TestSchema(c *C) { store := testCreateStore(c, "test_schema") defer store.Close() - d := testNewDDL(context.Background(), nil, store, nil, nil, testLease) + d := newDDL( + context.Background(), + WithStore(store), + WithLease(testLease), + ) defer d.Stop() ctx := testNewContext(d) dbInfo := testSchemaInfo(c, d, "test") @@ -186,12 +190,20 @@ func (s *testSchemaSuite) TestSchemaWaitJob(c *C) { store := testCreateStore(c, "test_schema_wait") defer store.Close() - d1 := testNewDDL(context.Background(), nil, store, nil, nil, testLease) + d1 := newDDL( + context.Background(), + WithStore(store), + WithLease(testLease), + ) defer d1.Stop() testCheckOwner(c, d1, true) - d2 := testNewDDL(context.Background(), nil, store, nil, nil, testLease*4) + d2 := newDDL( + context.Background(), + WithStore(store), + WithLease(testLease*4), + ) defer d2.Stop() ctx := testNewContext(d2) @@ -240,7 +252,11 @@ func (s *testSchemaSuite) TestSchemaResume(c *C) { store := testCreateStore(c, "test_schema_resume") defer store.Close() - d1 := testNewDDL(context.Background(), nil, store, nil, nil, testLease) + d1 := newDDL( + context.Background(), + WithStore(store), + WithLease(testLease), + ) defer d1.Stop() testCheckOwner(c, d1, true) diff --git a/ddl/stat_test.go b/ddl/stat_test.go index a08313be4813f..37a60d8a8df72 100644 --- a/ddl/stat_test.go +++ b/ddl/stat_test.go @@ -44,7 +44,11 @@ func (s *testStatSuite) TestStat(c *C) { store := testCreateStore(c, "test_stat") defer store.Close() - d := testNewDDL(context.Background(), nil, store, nil, nil, testLease) + d := newDDL( + context.Background(), + WithStore(store), + WithLease(testLease), + ) defer d.Stop() time.Sleep(testLease) diff --git a/ddl/table_test.go b/ddl/table_test.go index 335893cbe0c9b..cabb81eab7f18 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -308,7 +308,11 @@ func testGetTableWithError(d *ddl, schemaID, tableID int64) (table.Table, error) func (s *testTableSuite) SetUpSuite(c *C) { s.store = testCreateStore(c, "test_table") - s.d = testNewDDL(context.Background(), nil, s.store, nil, nil, testLease) + s.d = newDDL( + context.Background(), + WithStore(s.store), + WithLease(testLease), + ) s.dbInfo = testSchemaInfo(c, s.d, "test") testCreateSchema(c, testNewContext(s.d), s.d, s.dbInfo) diff --git a/ddl/util/syncer_test.go b/ddl/util/syncer_test.go index 9199ba2ac2857..6a7b8789bd2a8 100644 --- a/ddl/util/syncer_test.go +++ b/ddl/util/syncer_test.go @@ -55,7 +55,12 @@ func TestSyncerSimple(t *testing.T) { defer clus.Terminate(t) cli := clus.RandClient() ctx := goctx.Background() - d := NewDDL(ctx, cli, store, nil, nil, testLease, nil) + d := NewDDL( + ctx, + WithEtcdClient(cli), + WithStore(store), + WithLease(testLease), + ) defer d.Stop() // for init function @@ -82,7 +87,12 @@ func TestSyncerSimple(t *testing.T) { t.Fatalf("client get global version result not match, err %v", err) } - d1 := NewDDL(ctx, cli, store, nil, nil, testLease, nil) + d1 := NewDDL( + ctx, + WithEtcdClient(cli), + WithStore(store), + WithLease(testLease), + ) defer d1.Stop() if err = d1.SchemaSyncer().Init(ctx); err != nil { t.Fatalf("schema version syncer init failed %v", err) diff --git a/domain/domain.go b/domain/domain.go index 4d08582adc6da..f3fa3578eab6d 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -633,7 +633,15 @@ func (do *Domain) Init(ddlLease time.Duration, sysFactory func(*Domain) (pools.R ctx := context.Background() callback := &ddlCallback{do: do} d := do.ddl - do.ddl = ddl.NewDDL(ctx, do.etcdClient, do.store, do.infoHandle, callback, ddlLease, sysCtxPool) + do.ddl = ddl.NewDDL( + ctx, + ddl.WithEtcdClient(do.etcdClient), + ddl.WithStore(do.store), + ddl.WithInfoHandle(do.infoHandle), + ddl.WithHook(callback), + ddl.WithLease(ddlLease), + ddl.WithResourcePool(sysCtxPool), + ) failpoint.Inject("MockReplaceDDL", func(val failpoint.Value) { if val.(bool) { if err := do.ddl.Stop(); err != nil { diff --git a/domain/domain_test.go b/domain/domain_test.go index 8353330c2dc2e..a82bcb41e655b 100644 --- a/domain/domain_test.go +++ b/domain/domain_test.go @@ -92,7 +92,13 @@ func TestInfo(t *testing.T) { dom.etcdClient = cli // Mock new DDL and init the schema syncer with etcd client. goCtx := context.Background() - dom.ddl = ddl.NewDDL(goCtx, dom.GetEtcdClient(), s, dom.infoHandle, nil, ddlLease, nil) + dom.ddl = ddl.NewDDL( + goCtx, + ddl.WithEtcdClient(dom.GetEtcdClient()), + ddl.WithStore(s), + ddl.WithInfoHandle(dom.infoHandle), + ddl.WithLease(ddlLease), + ) err = failpoint.Enable("github.com/pingcap/tidb/domain/MockReplaceDDL", `return(true)`) if err != nil { t.Fatal(err) diff --git a/owner/manager_test.go b/owner/manager_test.go index a83e4dc699352..7d986b3cb2721 100644 --- a/owner/manager_test.go +++ b/owner/manager_test.go @@ -58,7 +58,12 @@ func TestSingle(t *testing.T) { defer clus.Terminate(t) cli := clus.RandClient() ctx := goctx.Background() - d := NewDDL(ctx, cli, store, nil, nil, testLease, nil) + d := NewDDL( + ctx, + WithEtcdClient(cli), + WithStore(store), + WithLease(testLease), + ) defer d.Stop() isOwner := checkOwner(d, true) @@ -109,13 +114,23 @@ func TestCluster(t *testing.T) { defer clus.Terminate(t) cli := clus.Client(0) - d := NewDDL(goctx.Background(), cli, store, nil, nil, testLease, nil) + d := NewDDL( + goctx.Background(), + WithEtcdClient(cli), + WithStore(store), + WithLease(testLease), + ) isOwner := checkOwner(d, true) if !isOwner { t.Fatalf("expect true, got isOwner:%v", isOwner) } cli1 := clus.Client(1) - d1 := NewDDL(goctx.Background(), cli1, store, nil, nil, testLease, nil) + d1 := NewDDL( + goctx.Background(), + WithEtcdClient(cli1), + WithStore(store), + WithLease(testLease), + ) isOwner = checkOwner(d1, false) if isOwner { t.Fatalf("expect false, got isOwner:%v", isOwner) @@ -135,7 +150,12 @@ func TestCluster(t *testing.T) { // d3 (not owner) stop cli3 := clus.Client(3) - d3 := NewDDL(goctx.Background(), cli3, store, nil, nil, testLease, nil) + d3 := NewDDL( + goctx.Background(), + WithEtcdClient(cli3), + WithStore(store), + WithLease(testLease), + ) defer d3.Stop() isOwner = checkOwner(d3, false) if isOwner { From 2aa3e27a6d535046efe789513d2fb60b474c5900 Mon Sep 17 00:00:00 2001 From: Lonng Date: Mon, 2 Sep 2019 11:01:08 +0800 Subject: [PATCH 09/21] infoschema,store: fix the inconsistent definition of StoreStatus between TiDB/PD (#11969) --- infoschema/tables.go | 8 ++++---- store/helper/helper.go | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/infoschema/tables.go b/infoschema/tables.go index 7cf20c741c632..31b1c00cd450a 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -594,8 +594,8 @@ var tableTiKVStoreStatusCols = []columnInfo{ {"LEADER_SCORE", mysql.TypeLonglong, 21, 0, nil, nil}, {"LEADER_SIZE", mysql.TypeLonglong, 21, 0, nil, nil}, {"REGION_COUNT", mysql.TypeLonglong, 21, 0, nil, nil}, - {"REGION_WEIGHT", mysql.TypeLonglong, 21, 0, nil, nil}, - {"REGION_SCORE", mysql.TypeLonglong, 21, 0, nil, nil}, + {"REGION_WEIGHT", mysql.TypeDouble, 22, 0, nil, nil}, + {"REGION_SCORE", mysql.TypeDouble, 22, 0, nil, nil}, {"REGION_SIZE", mysql.TypeLonglong, 21, 0, nil, nil}, {"START_TS", mysql.TypeDatetime, 0, 0, nil, nil}, {"LAST_HEARTBEAT_TS", mysql.TypeDatetime, 0, 0, nil, nil}, @@ -822,8 +822,8 @@ func dataForTiKVStoreStatus(ctx sessionctx.Context) (records [][]types.Datum, er row[10].SetInt64(storeStat.Status.LeaderScore) row[11].SetInt64(storeStat.Status.LeaderSize) row[12].SetInt64(storeStat.Status.RegionCount) - row[13].SetInt64(storeStat.Status.RegionWeight) - row[14].SetInt64(storeStat.Status.RegionScore) + row[13].SetFloat64(storeStat.Status.RegionWeight) + row[14].SetFloat64(storeStat.Status.RegionScore) row[15].SetInt64(storeStat.Status.RegionSize) startTs := types.Time{ Time: types.FromGoTime(storeStat.Status.StartTs), diff --git a/store/helper/helper.go b/store/helper/helper.go index 7ef5e612777d5..2cceec180d52f 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -661,8 +661,8 @@ type StoreDetailStat struct { LeaderScore int64 `json:"leader_score"` LeaderSize int64 `json:"leader_size"` RegionCount int64 `json:"region_count"` - RegionWeight int64 `json:"region_weight"` - RegionScore int64 `json:"region_score"` + RegionWeight float64 `json:"region_weight"` + RegionScore float64 `json:"region_score"` RegionSize int64 `json:"region_size"` StartTs time.Time `json:"start_ts"` LastHeartbeatTs time.Time `json:"last_heartbeat_ts"` From ef104a89b88c1f7f33b57b2dc49923488cb4518f Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 2 Sep 2019 11:41:05 +0800 Subject: [PATCH 10/21] distsql: merge executioin detail in stream cop (#11938) --- distsql/stream.go | 1 + 1 file changed, 1 insertion(+) diff --git a/distsql/stream.go b/distsql/stream.go index 78a88bfaa655a..42ed3506e09e4 100644 --- a/distsql/stream.go +++ b/distsql/stream.go @@ -98,6 +98,7 @@ func (r *streamResult) readDataFromResponse(ctx context.Context, resp kv.Respons } r.feedback.Update(resultSubset.GetStartKey(), stream.OutputCounts) r.partialCount++ + r.ctx.GetSessionVars().StmtCtx.MergeExecDetails(resultSubset.GetExecDetails(), nil) return false, nil } From b239f2f04d3aa19a732356b16e12796806f5f2d4 Mon Sep 17 00:00:00 2001 From: "Zhuomin(Charming) Liu" Date: Mon, 2 Sep 2019 16:16:05 +0800 Subject: [PATCH 11/21] planner: split avg to count and sum for TableReader cop task (#11926) --- cmd/explaintest/r/tpch.result | 4 ++-- planner/core/physical_plan_test.go | 2 +- planner/core/task.go | 29 +++++++++++++++++++++++++++++ 3 files changed, 32 insertions(+), 3 deletions(-) diff --git a/cmd/explaintest/r/tpch.result b/cmd/explaintest/r/tpch.result index e860fedcd012b..754a35d42720b 100644 --- a/cmd/explaintest/r/tpch.result +++ b/cmd/explaintest/r/tpch.result @@ -123,7 +123,7 @@ Sort_6 2.94 root tpch.lineitem.l_returnflag:asc, tpch.lineitem.l_linestatus:asc └─Projection_8 2.94 root tpch.lineitem.l_returnflag, tpch.lineitem.l_linestatus, 3_col_0, 3_col_1, 3_col_2, 3_col_3, 3_col_4, 3_col_5, 3_col_6, 3_col_7 └─HashAgg_14 2.94 root group by:col_13, col_14, funcs:sum(col_0), sum(col_1), sum(col_2), sum(col_3), avg(col_4, col_5), avg(col_6, col_7), avg(col_8, col_9), count(col_10), firstrow(col_13), firstrow(col_14) └─TableReader_15 2.94 root data:HashAgg_9 - └─HashAgg_9 2.94 cop group by:tpch.lineitem.l_linestatus, tpch.lineitem.l_returnflag, funcs:sum(tpch.lineitem.l_quantity), sum(tpch.lineitem.l_extendedprice), sum(mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount))), sum(mul(mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)), plus(1, tpch.lineitem.l_tax))), avg(tpch.lineitem.l_quantity), avg(tpch.lineitem.l_extendedprice), avg(tpch.lineitem.l_discount), count(1) + └─HashAgg_9 2.94 cop group by:tpch.lineitem.l_linestatus, tpch.lineitem.l_returnflag, funcs:sum(tpch.lineitem.l_quantity), sum(tpch.lineitem.l_extendedprice), sum(mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount))), sum(mul(mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)), plus(1, tpch.lineitem.l_tax))), count(tpch.lineitem.l_quantity), sum(tpch.lineitem.l_quantity), count(tpch.lineitem.l_extendedprice), sum(tpch.lineitem.l_extendedprice), count(tpch.lineitem.l_discount), sum(tpch.lineitem.l_discount), count(1) └─Selection_13 293795345.00 cop le(tpch.lineitem.l_shipdate, 1998-08-15) └─TableScan_12 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false /* @@ -981,7 +981,7 @@ Projection_16 1.00 root div(11_col_0, 7.0) │ └─TableScan_31 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false └─HashAgg_40 9943040.00 root group by:col_3, funcs:avg(col_0, col_1), firstrow(col_3) └─TableReader_41 9943040.00 root data:HashAgg_36 - └─HashAgg_36 9943040.00 cop group by:tpch.lineitem.l_partkey, funcs:avg(tpch.lineitem.l_quantity) + └─HashAgg_36 9943040.00 cop group by:tpch.lineitem.l_partkey, funcs:count(tpch.lineitem.l_quantity), sum(tpch.lineitem.l_quantity) └─TableScan_39 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false /* Q18 Large Volume Customer Query diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index 74ba9ea2b1fb6..da94c122fbb73 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -864,7 +864,7 @@ func (s *testPlanSuite) TestDAGPlanBuilderAgg(c *C) { // Test agg + table. { sql: "select sum(a), avg(b + c) from t group by d", - best: "TableReader(Table(t)->HashAgg)->HashAgg", + best: "TableReader(Table(t))->Projection->HashAgg", }, { sql: "select sum(distinct a), avg(b + c) from t group by d", diff --git a/planner/core/task.go b/planner/core/task.go index a56f757216c29..80dd85043dfe6 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -344,6 +344,34 @@ func (p *PhysicalMergeJoin) attach2Task(tasks ...task) task { } } +// splitCopAvg2CountAndSum splits the cop avg function to count and sum. +// Now it's only used for TableReader. +func splitCopAvg2CountAndSum(p PhysicalPlan) { + var baseAgg *basePhysicalAgg + if agg, ok := p.(*PhysicalStreamAgg); ok { + baseAgg = &agg.basePhysicalAgg + } + if agg, ok := p.(*PhysicalHashAgg); ok { + baseAgg = &agg.basePhysicalAgg + } + if baseAgg == nil { + return + } + for i := len(baseAgg.AggFuncs) - 1; i >= 0; i-- { + f := baseAgg.AggFuncs[i] + if f.Name == ast.AggFuncAvg { + sumAgg := *f + sumAgg.Name = ast.AggFuncSum + sumAgg.RetTp = baseAgg.Schema().Columns[i+1].RetType + cntAgg := *f + cntAgg.Name = ast.AggFuncCount + cntAgg.RetTp = baseAgg.Schema().Columns[i].RetType + cntAgg.RetTp.Flag = f.RetTp.Flag + baseAgg.AggFuncs = append(baseAgg.AggFuncs[:i], append([]*aggregation.AggFuncDesc{&cntAgg, &sumAgg}, baseAgg.AggFuncs[i+1:]...)...) + } + } +} + // finishCopTask means we close the coprocessor task and create a root task. func finishCopTask(ctx sessionctx.Context, task task) task { t, ok := task.(*copTask) @@ -413,6 +441,7 @@ func finishCopTask(ctx sessionctx.Context, task task) task { p.stats = t.indexPlan.statsInfo() newTask.p = p } else { + splitCopAvg2CountAndSum(t.tablePlan) p := PhysicalTableReader{tablePlan: t.tablePlan}.Init(ctx) p.stats = t.tablePlan.statsInfo() newTask.p = p From 304619a18421b08b8d337f37fc4f610122fc78a5 Mon Sep 17 00:00:00 2001 From: Shenghui Wu <793703860@qq.com> Date: Mon, 2 Sep 2019 16:45:35 +0800 Subject: [PATCH 12/21] executor: fix execution info of explain analyze (#11967) --- executor/distsql.go | 9 ++------ executor/executor.go | 2 +- executor/explain.go | 2 +- executor/explain_test.go | 47 ++++++++++++++++++++++++++++++++++++++++ executor/join.go | 2 +- 5 files changed, 52 insertions(+), 10 deletions(-) diff --git a/executor/distsql.go b/executor/distsql.go index ffb28e46fb784..6f5d5f2bcffda 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -22,7 +22,6 @@ import ( "strconv" "sync" "sync/atomic" - "time" "unsafe" "github.com/pingcap/errors" @@ -566,10 +565,6 @@ func (e *IndexLookUpExecutor) Close() error { // Next implements Exec Next interface. func (e *IndexLookUpExecutor) Next(ctx context.Context, req *chunk.Chunk) error { - if e.runtimeStats != nil { - start := time.Now() - defer func() { e.runtimeStats.Record(time.Since(start), req.NumRows()) }() - } if !e.workerStarted { if err := e.startWorkers(ctx, req.RequiredRows()); err != nil { return err @@ -801,7 +796,7 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta tblInfo := w.idxLookup.table.Meta() vals := make([]types.Datum, 0, len(w.idxTblCols)) for { - err := tableReader.Next(ctx, chk) + err := Next(ctx, tableReader, chk) if err != nil { return errors.Trace(err) } @@ -878,7 +873,7 @@ func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) er task.rows = make([]chunk.Row, 0, handleCnt) for { chk := newFirstChunk(tableReader) - err = tableReader.Next(ctx, chk) + err = Next(ctx, tableReader, chk) if err != nil { logutil.Logger(ctx).Error("table reader fetch next chunk failed", zap.Error(err)) return err diff --git a/executor/executor.go b/executor/executor.go index 2b258f5669cba..3e2548ec081ef 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -522,7 +522,7 @@ func (e *CheckTableExec) checkIndexHandle(ctx context.Context, num int, src *Ind var err error for { - err = src.Next(ctx, chk) + err = Next(ctx, src, chk) if err != nil { break } diff --git a/executor/explain.go b/executor/explain.go index 8dca1e894ba41..2e4c8103faa4c 100644 --- a/executor/explain.go +++ b/executor/explain.go @@ -74,7 +74,7 @@ func (e *ExplainExec) generateExplainInfo(ctx context.Context) ([][]string, erro if e.analyzeExec != nil { chk := newFirstChunk(e.analyzeExec) for { - err := e.analyzeExec.Next(ctx, chk) + err := Next(ctx, e.analyzeExec, chk) if err != nil { return nil, err } diff --git a/executor/explain_test.go b/executor/explain_test.go index 4af5141e71779..89aaa2810e637 100644 --- a/executor/explain_test.go +++ b/executor/explain_test.go @@ -152,3 +152,50 @@ func (s *testSuite1) checkMemoryInfo(c *C, tk *testkit.TestKit, sql string) { } } } + +func (s *testSuite1) TestExplainAnalyzeExecutionInfo(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (v int, k int, key(k))") + tk.MustExec("insert into t values (1, 1), (1, 1), (1, 1), (1, 1), (1, 1)") + + s.checkExecutionInfo(c, tk, "explain analyze select * from t order by v") + s.checkExecutionInfo(c, tk, "explain analyze select * from t order by v limit 5") + s.checkExecutionInfo(c, tk, "explain analyze select /*+ TIDB_HJ(t1, t2) */ t1.k from t t1, t t2 where t1.v = t2.v+1") + s.checkExecutionInfo(c, tk, "explain analyze select /*+ TIDB_SMJ(t1, t2) */ t1.k from t t1, t t2 where t1.k = t2.k+1") + s.checkExecutionInfo(c, tk, "explain analyze select /*+ TIDB_INLJ(t1, t2) */ t1.k from t t1, t t2 where t1.k = t2.k and t1.v=1") + s.checkExecutionInfo(c, tk, "explain analyze select sum(k) from t group by v") + s.checkExecutionInfo(c, tk, "explain analyze select sum(v) from t group by k") + s.checkExecutionInfo(c, tk, "explain analyze select * from t") + s.checkExecutionInfo(c, tk, "explain analyze select k from t use index(k)") + s.checkExecutionInfo(c, tk, "explain analyze select * from t use index(k)") + + tk.MustExec("CREATE TABLE IF NOT EXISTS nation ( N_NATIONKEY BIGINT NOT NULL,N_NAME CHAR(25) NOT NULL,N_REGIONKEY BIGINT NOT NULL,N_COMMENT VARCHAR(152),PRIMARY KEY (N_NATIONKEY));") + tk.MustExec("CREATE TABLE IF NOT EXISTS part ( P_PARTKEY BIGINT NOT NULL,P_NAME VARCHAR(55) NOT NULL,P_MFGR CHAR(25) NOT NULL,P_BRAND CHAR(10) NOT NULL,P_TYPE VARCHAR(25) NOT NULL,P_SIZE BIGINT NOT NULL,P_CONTAINER CHAR(10) NOT NULL,P_RETAILPRICE DECIMAL(15,2) NOT NULL,P_COMMENT VARCHAR(23) NOT NULL,PRIMARY KEY (P_PARTKEY));") + tk.MustExec("CREATE TABLE IF NOT EXISTS supplier ( S_SUPPKEY BIGINT NOT NULL,S_NAME CHAR(25) NOT NULL,S_ADDRESS VARCHAR(40) NOT NULL,S_NATIONKEY BIGINT NOT NULL,S_PHONE CHAR(15) NOT NULL,S_ACCTBAL DECIMAL(15,2) NOT NULL,S_COMMENT VARCHAR(101) NOT NULL,PRIMARY KEY (S_SUPPKEY),CONSTRAINT FOREIGN KEY SUPPLIER_FK1 (S_NATIONKEY) references nation(N_NATIONKEY));") + tk.MustExec("CREATE TABLE IF NOT EXISTS partsupp ( PS_PARTKEY BIGINT NOT NULL,PS_SUPPKEY BIGINT NOT NULL,PS_AVAILQTY BIGINT NOT NULL,PS_SUPPLYCOST DECIMAL(15,2) NOT NULL,PS_COMMENT VARCHAR(199) NOT NULL,PRIMARY KEY (PS_PARTKEY,PS_SUPPKEY),CONSTRAINT FOREIGN KEY PARTSUPP_FK1 (PS_SUPPKEY) references supplier(S_SUPPKEY),CONSTRAINT FOREIGN KEY PARTSUPP_FK2 (PS_PARTKEY) references part(P_PARTKEY));") + tk.MustExec("CREATE TABLE IF NOT EXISTS orders ( O_ORDERKEY BIGINT NOT NULL,O_CUSTKEY BIGINT NOT NULL,O_ORDERSTATUS CHAR(1) NOT NULL,O_TOTALPRICE DECIMAL(15,2) NOT NULL,O_ORDERDATE DATE NOT NULL,O_ORDERPRIORITY CHAR(15) NOT NULL,O_CLERK CHAR(15) NOT NULL,O_SHIPPRIORITY BIGINT NOT NULL,O_COMMENT VARCHAR(79) NOT NULL,PRIMARY KEY (O_ORDERKEY),CONSTRAINT FOREIGN KEY ORDERS_FK1 (O_CUSTKEY) references customer(C_CUSTKEY));") + tk.MustExec("CREATE TABLE IF NOT EXISTS lineitem ( L_ORDERKEY BIGINT NOT NULL,L_PARTKEY BIGINT NOT NULL,L_SUPPKEY BIGINT NOT NULL,L_LINENUMBER BIGINT NOT NULL,L_QUANTITY DECIMAL(15,2) NOT NULL,L_EXTENDEDPRICE DECIMAL(15,2) NOT NULL,L_DISCOUNT DECIMAL(15,2) NOT NULL,L_TAX DECIMAL(15,2) NOT NULL,L_RETURNFLAG CHAR(1) NOT NULL,L_LINESTATUS CHAR(1) NOT NULL,L_SHIPDATE DATE NOT NULL,L_COMMITDATE DATE NOT NULL,L_RECEIPTDATE DATE NOT NULL,L_SHIPINSTRUCT CHAR(25) NOT NULL,L_SHIPMODE CHAR(10) NOT NULL,L_COMMENT VARCHAR(44) NOT NULL,PRIMARY KEY (L_ORDERKEY,L_LINENUMBER),CONSTRAINT FOREIGN KEY LINEITEM_FK1 (L_ORDERKEY) references orders(O_ORDERKEY),CONSTRAINT FOREIGN KEY LINEITEM_FK2 (L_PARTKEY,L_SUPPKEY) references partsupp(PS_PARTKEY, PS_SUPPKEY));") + + s.checkExecutionInfo(c, tk, "select nation, o_year, sum(amount) as sum_profit from ( select n_name as nation, extract(year from o_orderdate) as o_year, l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount from part, supplier, lineitem, partsupp, orders, nation where s_suppkey = l_suppkey and ps_suppkey = l_suppkey and ps_partkey = l_partkey and p_partkey = l_partkey and o_orderkey = l_orderkey and s_nationkey = n_nationkey and p_name like '%dim%' ) as profit group by nation, o_year order by nation, o_year desc;") + + tk.MustExec("drop table if exists nation") + tk.MustExec("drop table if exists part") + tk.MustExec("drop table if exists supplier") + tk.MustExec("drop table if exists partsupp") + tk.MustExec("drop table if exists orders") + tk.MustExec("drop table if exists lineitem") +} + +func (s *testSuite1) checkExecutionInfo(c *C, tk *testkit.TestKit, sql string) { + executionInfoCol := 4 + rows := tk.MustQuery(sql).Rows() + for _, row := range rows { + strs := make([]string, len(row)) + for i, c := range row { + strs[i] = c.(string) + } + + c.Assert(strs[executionInfoCol], Not(Equals), "time:0s, loops:0, rows:0") + } +} diff --git a/executor/join.go b/executor/join.go index 6c1d4b3056c19..d626451f74871 100644 --- a/executor/join.go +++ b/executor/join.go @@ -266,7 +266,7 @@ func (e *HashJoinExec) fetchInnerRows(ctx context.Context, chkCh chan<- *chunk.C return } chk := chunk.NewChunkWithCapacity(e.innerExec.base().retFieldTypes, e.ctx.GetSessionVars().MaxChunkSize) - err = e.innerExec.Next(ctx, chk) + err = Next(ctx, e.innerExec, chk) if err != nil { e.innerFinished <- errors.Trace(err) return From 2a155f4eafb43169391dc4bfc7f4180fc3fedb25 Mon Sep 17 00:00:00 2001 From: Mingcong Han Date: Mon, 2 Sep 2019 17:03:06 +0800 Subject: [PATCH 13/21] planner: use ExtraHandleColumn as PK when generating TablePath (#11959) * use ExtraHandleColumn as PK * fix fmt * move unittest to explaintest --- cmd/explaintest/r/access_path_selection.result | 13 +++++++++++++ cmd/explaintest/t/access_path_selection.test | 3 +++ planner/core/logical_plans.go | 6 ++++++ 3 files changed, 22 insertions(+) diff --git a/cmd/explaintest/r/access_path_selection.result b/cmd/explaintest/r/access_path_selection.result index c66fb99a5ea22..b5f9442bbede9 100644 --- a/cmd/explaintest/r/access_path_selection.result +++ b/cmd/explaintest/r/access_path_selection.result @@ -23,3 +23,16 @@ id count task operator info IndexReader_11 1104.45 root index:Selection_10 └─Selection_10 1104.45 cop lt(test.access_path_selection.b, 3) └─IndexScan_9 3323.33 cop table:access_path_selection, index:a, b, range:[-inf,3), keep order:false, stats:pseudo +explain select a, b from access_path_selection where a > 10 order by _tidb_rowid; +id count task operator info +Projection_6 3333.33 root test.access_path_selection.a, test.access_path_selection.b +└─TableReader_13 3333.33 root data:Selection_12 + └─Selection_12 3333.33 cop gt(test.access_path_selection.a, 10) + └─TableScan_11 10000.00 cop table:access_path_selection, range:[-inf,+inf], keep order:true, stats:pseudo +explain select max(_tidb_rowid) from access_path_selection; +id count task operator info +StreamAgg_13 1.00 root funcs:max(test.access_path_selection._tidb_rowid) +└─Limit_17 1.00 root offset:0, count:1 + └─TableReader_27 1.00 root data:Limit_26 + └─Limit_26 1.00 cop offset:0, count:1 + └─TableScan_25 1.25 cop table:access_path_selection, range:[-inf,+inf], keep order:true, desc, stats:pseudo diff --git a/cmd/explaintest/t/access_path_selection.test b/cmd/explaintest/t/access_path_selection.test index fef9118ca606d..96d2018944791 100644 --- a/cmd/explaintest/t/access_path_selection.test +++ b/cmd/explaintest/t/access_path_selection.test @@ -12,3 +12,6 @@ explain select a, b from access_path_selection where a < 3; # In this query, IDX_ab can't be used, so IDX_b is the best. explain select a, b from access_path_selection where b < 3; explain select a, b from access_path_selection where a < 3 and b < 3; +# _tidb_rowid should also be considered as PK. +explain select a, b from access_path_selection where a > 10 order by _tidb_rowid; +explain select max(_tidb_rowid) from access_path_selection; diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 8f1da5e54e653..4b6f4270b7cd1 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -631,6 +631,12 @@ func isColEqCorColOrConstant(filter expression.Expression, col *expression.Colum func (ds *DataSource) getPKIsHandleCol() *expression.Column { if !ds.tableInfo.PKIsHandle { + // If the PKIsHandle is false, return the ExtraHandleColumn. + for i, col := range ds.Columns { + if col.ID == model.ExtraHandleID { + return ds.schema.Columns[i] + } + } return nil } for i, col := range ds.Columns { From cfef1bc52fbffa02571c052d7c504b28cf3cd0af Mon Sep 17 00:00:00 2001 From: Tanner Date: Mon, 2 Sep 2019 17:19:31 +0800 Subject: [PATCH 14/21] planner/core: add a range check against display width of BIT column type (#11942) --- go.mod | 2 +- go.sum | 4 ++-- planner/core/integration_test.go | 21 +++++++++++++++++++++ planner/core/preprocess.go | 9 ++++++++- types/errors.go | 5 ++++- 5 files changed, 36 insertions(+), 5 deletions(-) diff --git a/go.mod b/go.mod index 6413c20dd1b09..24ebbadf68552 100644 --- a/go.mod +++ b/go.mod @@ -41,7 +41,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e github.com/pingcap/kvproto v0.0.0-20190724165112-ec9df5f208a7 github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 - github.com/pingcap/parser v0.0.0-20190828080649-a621a0f9b06c + github.com/pingcap/parser v0.0.0-20190902030720-275a827cf4e3 github.com/pingcap/pd v0.0.0-20190712044914-75a1f9f3062b github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible github.com/pingcap/tipb v0.0.0-20190806070524-16909e03435e diff --git a/go.sum b/go.sum index 87f81d487e605..855f4dd8adb93 100644 --- a/go.sum +++ b/go.sum @@ -165,8 +165,8 @@ github.com/pingcap/kvproto v0.0.0-20190724165112-ec9df5f208a7/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-20190828080649-a621a0f9b06c h1:VcQ60QkH7Vv98dbedNKDEO7RfUw+6TE59r/lQokNz/k= -github.com/pingcap/parser v0.0.0-20190828080649-a621a0f9b06c/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= +github.com/pingcap/parser v0.0.0-20190902030720-275a827cf4e3 h1:fRok+VIrzkDXHkVe7R1ByoLhsvfth/WHMBJmy/91/cM= +github.com/pingcap/parser v0.0.0-20190902030720-275a827cf4e3/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= github.com/pingcap/pd v0.0.0-20190712044914-75a1f9f3062b h1:oS9PftxQqgcRouKhhdaB52tXhVLEP7Ng3Qqsd6Z18iY= github.com/pingcap/pd v0.0.0-20190712044914-75a1f9f3062b/go.mod h1:3DlDlFT7EF64A1bmb/tulZb6wbPSagm5G4p1AlhaEDs= github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU= diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index ff7ca01e6dd92..2eae2b1a2a812 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -59,3 +59,24 @@ func (s *testIntegrationSuite) TestShowSubquery(c *C) { "a varchar(10) YES ", )) } + +func (s *testIntegrationSuite) BitColErrorMessage(c *C) { + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + tk := testkit.NewTestKit(c, store) + defer func() { + dom.Close() + store.Close() + }() + + tk.MustExec("use test") + tk.MustExec("drop table if exists bit_col_t") + tk.MustExec("create table bit_col_t (a bit(64))") + tk.MustExec("drop table bit_col_t") + tk.MustExec("create table bit_col_t (a bit(1))") + tk.MustExec("drop table bit_col_t") + _, err = tk.Exec("create table bit_col_t (a bit(0))") + c.Assert(err, NotNil) + _, err = tk.Exec("create table bit_col_t (a bit(65))") + c.Assert(err, NotNil) +} diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index d902ffba1f8e3..533195ce83c2f 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -628,13 +628,20 @@ func checkColumn(colDef *ast.ColumnDef) error { if tp.Flen > mysql.MaxDecimalWidth { return types.ErrTooBigPrecision.GenWithStackByArgs(tp.Flen, colDef.Name.Name.O, mysql.MaxDecimalWidth) } + case mysql.TypeBit: + if tp.Flen <= 0 { + return types.ErrInvalidFieldSize.GenWithStackByArgs(colDef.Name.Name.O) + } + if tp.Flen > mysql.MaxBitDisplayWidth { + return types.ErrTooBigDisplayWidth.GenWithStackByArgs(colDef.Name.Name.O, mysql.MaxBitDisplayWidth) + } default: // TODO: Add more types. } return nil } -// isDefaultValNowSymFunc checks whether defaul value is a NOW() builtin function. +// isDefaultValNowSymFunc checks whether default value is a NOW() builtin function. func isDefaultValNowSymFunc(expr ast.ExprNode) bool { if funcCall, ok := expr.(*ast.FuncCallExpr); ok { // Default value NOW() is transformed to CURRENT_TIMESTAMP() in parser. diff --git a/types/errors.go b/types/errors.go index f58bbf34cf60e..2881595eb828b 100644 --- a/types/errors.go +++ b/types/errors.go @@ -34,7 +34,7 @@ var ( // ErrDivByZero is return when do division by 0. ErrDivByZero = terror.ClassTypes.New(codeDivByZero, "Division by 0") // ErrTooBigDisplayWidth is return when display width out of range for column. - ErrTooBigDisplayWidth = terror.ClassTypes.New(codeTooBigDisplayWidth, "Too Big Display width") + ErrTooBigDisplayWidth = terror.ClassTypes.New(codeTooBigDisplayWidth, mysql.MySQLErrName[mysql.ErrTooBigDisplaywidth]) // ErrTooBigFieldLength is return when column length too big for column. ErrTooBigFieldLength = terror.ClassTypes.New(codeTooBigFieldLength, "Too Big Field length") // ErrTooBigSet is returned when too many strings for column. @@ -49,6 +49,8 @@ var ( ErrBadNumber = terror.ClassTypes.New(codeBadNumber, "Bad Number") // ErrInvalidDefault is returned when meet a invalid default value. ErrInvalidDefault = parser_types.ErrInvalidDefault + // ErrInvalidFieldSize is returned when the precision of a column is out of range. + ErrInvalidFieldSize = terror.ClassTypes.New(codeInvalidFieldSize, mysql.MySQLErrName[mysql.ErrInvalidFieldSize]) // ErrCastAsSignedOverflow is returned when positive out-of-range integer, and convert to it's negative complement. ErrCastAsSignedOverflow = terror.ClassTypes.New(codeUnknown, msgCastAsSignedOverflow) // ErrCastNegIntAsUnsigned is returned when a negative integer be casted to an unsigned int. @@ -93,6 +95,7 @@ const ( codeTruncatedWrongValue = terror.ErrCode(mysql.ErrTruncatedWrongValue) codeUnknown = terror.ErrCode(mysql.ErrUnknown) codeInvalidDefault = terror.ErrCode(mysql.ErrInvalidDefault) + codeInvalidFieldSize = terror.ErrCode(mysql.ErrInvalidFieldSize) codeMBiggerThanD = terror.ErrCode(mysql.ErrMBiggerThanD) codeDataOutOfRange = terror.ErrCode(mysql.ErrWarnDataOutOfRange) codeDuplicatedValueInType = terror.ErrCode(mysql.ErrDuplicatedValueInType) From 728ed194c0669cb4effe6398de464c3ffbfb6608 Mon Sep 17 00:00:00 2001 From: Arenatlx Date: Mon, 2 Sep 2019 05:58:28 -0500 Subject: [PATCH 15/21] fix load data rowCount should added first before call colsToRow(). (#11980) --- executor/load_data.go | 4 +++- server/server_test.go | 4 ++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/executor/load_data.go b/executor/load_data.go index 3120e7543f227..4983422c35ba1 100644 --- a/executor/load_data.go +++ b/executor/load_data.go @@ -264,8 +264,10 @@ func (e *LoadDataInfo) InsertData(ctx context.Context, prevData, curData []byte) if err != nil { return nil, false, err } - e.colsToRow(ctx, cols) + // rowCount will be used in fillRow(), last insert ID will be assigned according to the rowCount = 1. + // So should add first here. e.rowCount++ + e.colsToRow(ctx, cols) e.curBatchCnt++ if e.maxRowsInBatch != 0 && e.rowCount%e.maxRowsInBatch == 0 { reachLimit = true diff --git a/server/server_test.go b/server/server_test.go index f23d2b0c793ed..c7ef21ae1a3b9 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -427,7 +427,7 @@ func runTestLoadData(c *C, server *Server) { dbt.Assert(err, IsNil) lastID, err = rs.LastInsertId() dbt.Assert(err, IsNil) - dbt.Assert(lastID, Equals, int64(7)) + dbt.Assert(lastID, Equals, int64(6)) affectedRows, err = rs.RowsAffected() dbt.Assert(err, IsNil) dbt.Assert(affectedRows, Equals, int64(4)) @@ -466,7 +466,7 @@ func runTestLoadData(c *C, server *Server) { dbt.Assert(err, IsNil) lastID, err = rs.LastInsertId() dbt.Assert(err, IsNil) - dbt.Assert(lastID, Equals, int64(11)) + dbt.Assert(lastID, Equals, int64(10)) affectedRows, err = rs.RowsAffected() dbt.Assert(err, IsNil) dbt.Assert(affectedRows, Equals, int64(799)) From 7372dcd6d2bf04e459dc1636f2f9836dc083c596 Mon Sep 17 00:00:00 2001 From: gaojingyu <1121087373@qq.com> Date: Mon, 2 Sep 2019 20:16:35 +0800 Subject: [PATCH 16/21] Planner:eliminate redundant projection (#11920) --- cmd/explaintest/r/explain_easy.result | 36 +++++++++---------- cmd/explaintest/r/partition_pruning.result | 10 +++--- cmd/explaintest/r/select.result | 32 ++++++++--------- cmd/explaintest/r/topn_pushdown.result | 9 +++-- planner/core/logical_plan_test.go | 40 ++++++++++++++++++---- planner/core/physical_plan_test.go | 4 +-- planner/core/rule_column_pruning.go | 10 ++++++ planner/core/rule_eliminate_projection.go | 23 +++++++++++++ 8 files changed, 109 insertions(+), 55 deletions(-) diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index ad46fe9a5c6dc..2b79925cadb5a 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -108,14 +108,13 @@ Projection_9 9990.00 root test.t1.c1 └─TableScan_18 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo explain select (select count(1) k from t1 s where s.c1 = t1.c1 having k != 0) from t1; id count task operator info -Projection_12 10000.00 root k -└─Projection_13 10000.00 root test.t1.c1, ifnull(5_col_0, 0) - └─MergeJoin_14 10000.00 root left outer join, left key:test.t1.c1, right key:test.s.c1 - ├─TableReader_17 10000.00 root data:TableScan_16 - │ └─TableScan_16 10000.00 cop table:t1, range:[-inf,+inf], keep order:true, stats:pseudo - └─Projection_18 8000.00 root 1, test.s.c1 - └─TableReader_20 10000.00 root data:TableScan_19 - └─TableScan_19 10000.00 cop table:s, range:[-inf,+inf], keep order:true, stats:pseudo +Projection_12 10000.00 root ifnull(5_col_0, 0) +└─MergeJoin_13 10000.00 root left outer join, left key:test.t1.c1, right key:test.s.c1 + ├─TableReader_16 10000.00 root data:TableScan_15 + │ └─TableScan_15 10000.00 cop table:t1, range:[-inf,+inf], keep order:true, stats:pseudo + └─Projection_17 8000.00 root 1, test.s.c1 + └─TableReader_19 10000.00 root data:TableScan_18 + └─TableScan_18 10000.00 cop table:s, range:[-inf,+inf], keep order:true, stats:pseudo explain select * from information_schema.columns; id count task operator info MemTableScan_4 10000.00 root @@ -648,22 +647,19 @@ id count task operator info Sort_13 2.00 root a:asc └─HashAgg_17 2.00 root group by:a, funcs:firstrow(join_agg_0) └─Union_18 2.00 root - ├─HashAgg_21 1.00 root group by:a, funcs:firstrow(a), firstrow(a) - │ └─Projection_22 1.00 root 0 - │ └─TableDual_23 1.00 root rows:1 - └─HashAgg_26 1.00 root group by:a, funcs:firstrow(a), firstrow(a) - └─Projection_27 1.00 root 1 - └─TableDual_28 1.00 root rows:1 + ├─HashAgg_19 1.00 root group by:0, funcs:firstrow(0), firstrow(0) + │ └─TableDual_22 1.00 root rows:1 + └─HashAgg_25 1.00 root group by:1, funcs:firstrow(1), firstrow(1) + └─TableDual_28 1.00 root rows:1 explain SELECT 0 AS a FROM dual UNION (SELECT 1 AS a FROM dual ORDER BY a); id count task operator info HashAgg_15 2.00 root group by:a, funcs:firstrow(join_agg_0) └─Union_16 2.00 root - ├─HashAgg_19 1.00 root group by:a, funcs:firstrow(a), firstrow(a) - │ └─Projection_20 1.00 root 0 - │ └─TableDual_21 1.00 root rows:1 - └─StreamAgg_26 1.00 root group by:a, funcs:firstrow(a), firstrow(a) - └─Projection_31 1.00 root 1 - └─TableDual_32 1.00 root rows:1 + ├─HashAgg_17 1.00 root group by:0, funcs:firstrow(0), firstrow(0) + │ └─TableDual_20 1.00 root rows:1 + └─StreamAgg_27 1.00 root group by:a, funcs:firstrow(a), firstrow(a) + └─Projection_32 1.00 root 1 + └─TableDual_33 1.00 root rows:1 create table t (i int key, j int, unique key (i, j)); begin; insert into t values (1, 1); diff --git a/cmd/explaintest/r/partition_pruning.result b/cmd/explaintest/r/partition_pruning.result index 216897925d8d4..6a7431da335d6 100644 --- a/cmd/explaintest/r/partition_pruning.result +++ b/cmd/explaintest/r/partition_pruning.result @@ -4010,11 +4010,11 @@ create table t1 (s1 int); explain select 1 from t1 union all select 2; id count task operator info Union_8 10001.00 root -├─Projection_10 10000.00 root 1 -│ └─TableReader_12 10000.00 root data:TableScan_11 -│ └─TableScan_11 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo -└─Projection_14 1.00 root 2 - └─TableDual_15 1.00 root rows:1 +├─Projection_9 10000.00 root 1 +│ └─TableReader_11 10000.00 root data:TableScan_10 +│ └─TableScan_10 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo +└─Projection_12 1.00 root 2 + └─TableDual_13 1.00 root rows:1 drop table t1; create table t1 (a int) partition by range(a) ( diff --git a/cmd/explaintest/r/select.result b/cmd/explaintest/r/select.result index c80b6cb8438ae..c8b7cae81560e 100644 --- a/cmd/explaintest/r/select.result +++ b/cmd/explaintest/r/select.result @@ -382,24 +382,24 @@ drop table if exists t; create table t(a int, b int); explain select a != any (select a from t t2) from t t1; id count task operator info -Projection_9 10000.00 root and(or(or(gt(col_count, 1), ne(test.t1.a, col_firstrow)), if(ne(agg_col_sum, 0), NULL, 0)), and(ne(agg_col_cnt, 0), if(isnull(test.t1.a), NULL, 1))) -└─HashLeftJoin_10 10000.00 root CARTESIAN inner join, inner:StreamAgg_17 - ├─TableReader_13 10000.00 root data:TableScan_12 - │ └─TableScan_12 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo - └─StreamAgg_17 1.00 root funcs:firstrow(col_0), count(distinct col_1), sum(col_2), count(1) - └─Projection_27 10000.00 root test.t2.a, test.t2.a, cast(isnull(test.t2.a)) - └─TableReader_24 10000.00 root data:TableScan_23 - └─TableScan_23 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo +Projection_8 10000.00 root and(or(or(gt(col_count, 1), ne(test.t1.a, col_firstrow)), if(ne(agg_col_sum, 0), NULL, 0)), and(ne(agg_col_cnt, 0), if(isnull(test.t1.a), NULL, 1))) +└─HashLeftJoin_9 10000.00 root CARTESIAN inner join, inner:StreamAgg_16 + ├─TableReader_12 10000.00 root data:TableScan_11 + │ └─TableScan_11 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo + └─StreamAgg_16 1.00 root funcs:firstrow(col_0), count(distinct col_1), sum(col_2), count(1) + └─Projection_26 10000.00 root test.t2.a, test.t2.a, cast(isnull(test.t2.a)) + └─TableReader_23 10000.00 root data:TableScan_22 + └─TableScan_22 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo explain select a = all (select a from t t2) from t t1; id count task operator info -Projection_9 10000.00 root or(and(and(le(col_count, 1), eq(test.t1.a, col_firstrow)), if(ne(agg_col_sum, 0), NULL, 1)), or(eq(agg_col_cnt, 0), if(isnull(test.t1.a), NULL, 0))) -└─HashLeftJoin_10 10000.00 root CARTESIAN inner join, inner:StreamAgg_17 - ├─TableReader_13 10000.00 root data:TableScan_12 - │ └─TableScan_12 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo - └─StreamAgg_17 1.00 root funcs:firstrow(col_0), count(distinct col_1), sum(col_2), count(1) - └─Projection_27 10000.00 root test.t2.a, test.t2.a, cast(isnull(test.t2.a)) - └─TableReader_24 10000.00 root data:TableScan_23 - └─TableScan_23 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo +Projection_8 10000.00 root or(and(and(le(col_count, 1), eq(test.t1.a, col_firstrow)), if(ne(agg_col_sum, 0), NULL, 1)), or(eq(agg_col_cnt, 0), if(isnull(test.t1.a), NULL, 0))) +└─HashLeftJoin_9 10000.00 root CARTESIAN inner join, inner:StreamAgg_16 + ├─TableReader_12 10000.00 root data:TableScan_11 + │ └─TableScan_11 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo + └─StreamAgg_16 1.00 root funcs:firstrow(col_0), count(distinct col_1), sum(col_2), count(1) + └─Projection_26 10000.00 root test.t2.a, test.t2.a, cast(isnull(test.t2.a)) + └─TableReader_23 10000.00 root data:TableScan_22 + └─TableScan_22 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo drop table if exists t; create table t(a int, b int); drop table if exists s; diff --git a/cmd/explaintest/r/topn_pushdown.result b/cmd/explaintest/r/topn_pushdown.result index 19457028e055c..85836fc48a63e 100644 --- a/cmd/explaintest/r/topn_pushdown.result +++ b/cmd/explaintest/r/topn_pushdown.result @@ -2,8 +2,7 @@ explain select * from ((select 4 as a) union all (select 33 as a)) tmp order by id count task operator info TopN_17 1.00 root tmp.a:desc, offset:0, count:1 └─Union_21 2.00 root - ├─Projection_22 1.00 root cast(a) - │ └─Projection_23 1.00 root 4 - │ └─TableDual_24 1.00 root rows:1 - └─Projection_26 1.00 root 33 - └─TableDual_27 1.00 root rows:1 + ├─Projection_22 1.00 root cast(4) + │ └─TableDual_23 1.00 root rows:1 + └─Projection_24 1.00 root 33 + └─TableDual_25 1.00 root rows:1 diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index e2a570d063c08..c182132fddfa0 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -1297,6 +1297,32 @@ func (s *testPlanSuite) TestColumnPruning(c *C) { } } +func (s *testPlanSuite) TestProjectionEliminater(c *C) { + defer testleak.AfterTest(c)() + tests := []struct { + sql string + best string + }{ + { + sql: "select 1+num from (select 1+a as num from t) t1;", + best: "DataScan(t)->Projection", + }, + } + + ctx := context.Background() + for ith, tt := range tests { + comment := Commentf("for %s", tt.sql) + stmt, err := s.ParseOneStmt(tt.sql, "", "") + c.Assert(err, IsNil, comment) + + p, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) + c.Assert(err, IsNil) + p, err = logicalOptimize(context.TODO(), flagBuildKeyInfo|flagPrunColumns|flagEliminateProjection, p.(LogicalPlan)) + c.Assert(err, IsNil) + c.Assert(ToString(p), Equals, tt.best, Commentf("for %s %d", tt.sql, ith)) + } +} + func (s *testPlanSuite) TestAllocID(c *C) { ctx := MockContext() pA := DataSource{}.Init(ctx) @@ -1599,7 +1625,7 @@ func (s *testPlanSuite) TestAggPrune(c *C) { }{ { sql: "select a, count(b) from t group by a", - best: "DataScan(t)->Projection->Projection", + best: "DataScan(t)->Projection", }, { sql: "select sum(b) from t group by c, d, e", @@ -1607,19 +1633,19 @@ func (s *testPlanSuite) TestAggPrune(c *C) { }, { sql: "select tt.a, sum(tt.b) from (select a, b from t) tt group by tt.a", - best: "DataScan(t)->Projection->Projection", + best: "DataScan(t)->Projection", }, { sql: "select count(1) from (select count(1), a as b from t group by a) tt group by b", - best: "DataScan(t)->Projection->Projection", + best: "DataScan(t)->Projection", }, { sql: "select a, count(b) from t group by a", - best: "DataScan(t)->Projection->Projection", + best: "DataScan(t)->Projection", }, { sql: "select a, count(distinct a, b) from t group by a", - best: "DataScan(t)->Projection->Projection", + best: "DataScan(t)->Projection", }, } @@ -1925,12 +1951,12 @@ func (s *testPlanSuite) TestUnion(c *C) { }, { sql: "select * from (select 1 as a union select 1 union all select 2) t order by a", - best: "UnionAll{UnionAll{Dual->Projection->Projection->Dual->Projection->Projection}->Aggr(firstrow(a))->Projection->Dual->Projection->Projection}->Projection->Sort", + best: "UnionAll{UnionAll{Dual->Projection->Dual->Projection}->Aggr(firstrow(a))->Projection->Dual->Projection}->Projection->Sort", err: false, }, { sql: "select * from (select 1 as a union select 1 union all select 2) t order by (select a)", - best: "Apply{UnionAll{UnionAll{Dual->Projection->Projection->Dual->Projection->Projection}->Aggr(firstrow(a))->Projection->Dual->Projection->Projection}->Dual->Projection->MaxOneRow}->Sort->Projection", + best: "Apply{UnionAll{UnionAll{Dual->Projection->Dual->Projection}->Aggr(firstrow(a))->Projection->Dual->Projection}->Dual->Projection->MaxOneRow}->Sort->Projection", err: false, }, } diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index da94c122fbb73..ecc5b63f0cffa 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -529,7 +529,7 @@ func (s *testPlanSuite) TestDAGPlanBuilderSubquery(c *C) { }, { sql: "select (select count(*) from t s, t t1 where s.a = t.a and s.a = t1.a) from t", - best: "LeftHashJoin{TableReader(Table(t))->MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(test.s.a,test.t1.a)->StreamAgg}(test.t.a,test.s.a)->Projection->Projection", + best: "LeftHashJoin{TableReader(Table(t))->MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(test.s.a,test.t1.a)->StreamAgg}(test.t.a,test.s.a)->Projection", }, { sql: "select (select count(*) from t s, t t1 where s.a = t.a and s.a = t1.a) from t order by t.a", @@ -912,7 +912,7 @@ func (s *testPlanSuite) TestDAGPlanBuilderAgg(c *C) { }, { sql: "select (select count(1) k from t s where s.a = t.a having k != 0) from t", - best: "MergeLeftOuterJoin{TableReader(Table(t))->TableReader(Table(t))->Projection}(test.t.a,test.s.a)->Projection->Projection", + best: "MergeLeftOuterJoin{TableReader(Table(t))->TableReader(Table(t))->Projection}(test.t.a,test.s.a)->Projection", }, // Test stream agg with multi group by columns. { diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index 38e25f0aeca34..e2dba4cfff9d7 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -53,6 +53,16 @@ func getUsedList(usedCols []*expression.Column, schema *expression.Schema) ([]bo return used, nil } +// exprsHasSideEffects checks if any of the expressions has side effects. +func exprsHasSideEffects(exprs []expression.Expression) bool { + for _, expr := range exprs { + if exprHasSetVarOrSleep(expr) { + return true + } + } + return false +} + // exprHasSetVarOrSleep checks if the expression has SetVar function or Sleep function. func exprHasSetVarOrSleep(expr expression.Expression) bool { scalaFunc, isScalaFunc := expr.(*expression.ScalarFunction) diff --git a/planner/core/rule_eliminate_projection.go b/planner/core/rule_eliminate_projection.go index 5e0f07c758d10..9d4e1b8be6a6f 100644 --- a/planner/core/rule_eliminate_projection.go +++ b/planner/core/rule_eliminate_projection.go @@ -137,6 +137,14 @@ func (pe *projectionEliminater) eliminate(p LogicalPlan, replace map[string]*exp } } p.replaceExprColumns(replace) + if isProj { + if child, ok := p.Children()[0].(*LogicalProjection); ok && !exprsHasSideEffects(child.Exprs) { + for i := range proj.Exprs { + proj.Exprs[i] = replaceColumnOfExpr(proj.Exprs[i], child) + } + p.Children()[0] = child.Children()[0] + } + } if !(isProj && canEliminate && canProjectionBeEliminatedLoose(proj)) { return p @@ -148,6 +156,21 @@ func (pe *projectionEliminater) eliminate(p LogicalPlan, replace map[string]*exp return p.Children()[0] } +func replaceColumnOfExpr(expr expression.Expression, proj *LogicalProjection) expression.Expression { + switch v := expr.(type) { + case *expression.Column: + idx := proj.Schema().ColumnIndex(v) + if idx != -1 && idx < len(proj.Exprs) { + return proj.Exprs[idx] + } + case *expression.ScalarFunction: + for i := range v.GetArgs() { + v.GetArgs()[i] = replaceColumnOfExpr(v.GetArgs()[i], proj) + } + } + return expr +} + func (p *LogicalJoin) replaceExprColumns(replace map[string]*expression.Column) { for _, equalExpr := range p.EqualConditions { resolveExprAndReplace(equalExpr, replace) From 67d4f913dd3f305b39c4d6ba53a6ff82fc040c39 Mon Sep 17 00:00:00 2001 From: Lonng Date: Mon, 2 Sep 2019 21:30:13 +0800 Subject: [PATCH 17/21] expression: fix type infer of unaryMinus which should return ETDecimal if ETInt overflow int (#11989) Signed-off-by: Lonng --- expression/builtin_op.go | 7 +++---- expression/integration_test.go | 6 ++++++ 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/expression/builtin_op.go b/expression/builtin_op.go index c7af218cdb231..096f817cc5ab5 100644 --- a/expression/builtin_op.go +++ b/expression/builtin_op.go @@ -712,16 +712,15 @@ func (c *unaryMinusFunctionClass) handleIntOverflow(arg *Constant) (overflow boo // typeInfer infers unaryMinus function return type. when the arg is an int constant and overflow, // typerInfer will infers the return type as types.ETDecimal, not types.ETInt. -func (c *unaryMinusFunctionClass) typeInfer(ctx sessionctx.Context, argExpr Expression) (types.EvalType, bool) { +func (c *unaryMinusFunctionClass) typeInfer(argExpr Expression) (types.EvalType, bool) { tp := argExpr.GetType().EvalType() if tp != types.ETInt && tp != types.ETDecimal { tp = types.ETReal } - sc := ctx.GetSessionVars().StmtCtx overflow := false // TODO: Handle float overflow. - if arg, ok := argExpr.(*Constant); sc.InSelectStmt && ok && tp == types.ETInt { + if arg, ok := argExpr.(*Constant); ok && tp == types.ETInt { overflow = c.handleIntOverflow(arg) if overflow { tp = types.ETDecimal @@ -736,7 +735,7 @@ func (c *unaryMinusFunctionClass) getFunction(ctx sessionctx.Context, args []Exp } argExpr, argExprTp := args[0], args[0].GetType() - _, intOverflow := c.typeInfer(ctx, argExpr) + _, intOverflow := c.typeInfer(argExpr) var bf baseBuiltinFunc switch argExprTp.EvalType() { diff --git a/expression/integration_test.go b/expression/integration_test.go index 7aad56f99c9bc..79d2352b96752 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -2098,6 +2098,12 @@ func (s *testIntegrationSuite) TestOpBuiltin(c *C) { // for unaryPlus result = tk.MustQuery(`select +1, +0, +(-9), +(-0.001), +0.999, +null, +"aaa"`) result.Check(testkit.Rows("1 0 -9 -0.001 0.999 aaa")) + // for unaryMinus + tk.MustExec("drop table if exists f") + tk.MustExec("create table f(a decimal(65,0))") + tk.MustExec("insert into f value (-17000000000000000000)") + result = tk.MustQuery("select a from f") + result.Check(testkit.Rows("-17000000000000000000")) } func (s *testIntegrationSuite) TestDatetimeOverflow(c *C) { From dbdb2f4331c0acd39b19f9c45b45cdcc6f1804ad Mon Sep 17 00:00:00 2001 From: Arenatlx Date: Mon, 2 Sep 2019 08:44:35 -0500 Subject: [PATCH 18/21] dml : fix last insert id when autoid specified by user in first row. (#11973) --- executor/insert_common.go | 5 +++-- executor/insert_test.go | 16 ++++++++++++++++ 2 files changed, 19 insertions(+), 2 deletions(-) diff --git a/executor/insert_common.go b/executor/insert_common.go index 7fba881845d35..b6d31577ab5ac 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -570,8 +570,9 @@ func (e *InsertValues) adjustAutoIncrementDatum(ctx context.Context, d types.Dat if e.filterErr(err) != nil { return types.Datum{}, err } - // It's compatible with mysql. So it sets last insert id to the first row. - if e.rowCount == 1 { + // It's compatible with mysql setting the first allocated autoID to lastInsertID. + // Cause autoID may be specified by user, judge only the first row is not suitable. + if e.lastInsertID == 0 { e.lastInsertID = uint64(recordID) } } diff --git a/executor/insert_test.go b/executor/insert_test.go index c5d63e86d919c..b9ac781e5ea93 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -353,6 +353,22 @@ func (s *testSuite3) TestInsertWithAutoidSchema(c *C) { `select * from t1 where id = 9`, testkit.Rows(`9 9`), }, + // test last insert id + { + `insert into t1 values(3000, -1), (null, -2)`, + `select * from t1 where id = 3000`, + testkit.Rows(`3000 -1`), + }, + { + `;`, + `select * from t1 where id = 3001`, + testkit.Rows(`3001 -2`), + }, + { + `;`, + `select last_insert_id()`, + testkit.Rows(`3001`), + }, { `insert into t2(id, n) values(1, 1)`, `select * from t2 where id = 1`, From 756e891ca61c6ff50e8302a438d13017e91586c9 Mon Sep 17 00:00:00 2001 From: satoru Date: Tue, 3 Sep 2019 08:08:35 +0800 Subject: [PATCH 19/21] table: Encode/Decode handle faster by reducing escaped variables (#11982) --- table/tables/index.go | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/table/tables/index.go b/table/tables/index.go index 570fb3c20095e..cb865ea5add66 100644 --- a/table/tables/index.go +++ b/table/tables/index.go @@ -35,20 +35,14 @@ import ( // EncodeHandle encodes handle in data. func EncodeHandle(h int64) []byte { - buf := &bytes.Buffer{} - err := binary.Write(buf, binary.BigEndian, h) - if err != nil { - panic(err) - } - return buf.Bytes() + var data [8]byte + binary.BigEndian.PutUint64(data[:], uint64(h)) + return data[:] } // DecodeHandle decodes handle in data. func DecodeHandle(data []byte) (int64, error) { - var h int64 - buf := bytes.NewBuffer(data) - err := binary.Read(buf, binary.BigEndian, &h) - return h, err + return int64(binary.BigEndian.Uint64(data)), nil } // indexIter is for KV store index iterator. From b90ebe11099f9a04c061b62bd1024cfc43e04f85 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Tue, 3 Sep 2019 09:41:05 +0800 Subject: [PATCH 20/21] expression: enable vectorized expression evaluation by default (#11965) --- executor/projection.go | 3 ++ expression/builtin_vectorized_test.go | 51 +++++++++++++++++++++++++++ expression/chunk_executor.go | 5 +-- expression/column.go | 16 +++++++++ expression/constant.go | 5 ++- expression/integration_test.go | 6 ++++ expression/vectorized.go | 49 +++++++++++++------------ sessionctx/variable/session.go | 1 + sessionctx/variable/tidb_vars.go | 2 +- util/chunk/column.go | 41 +++++++++++++-------- util/chunk/column_test.go | 30 ++++++++-------- 11 files changed, 150 insertions(+), 59 deletions(-) diff --git a/executor/projection.go b/executor/projection.go index be6aafeeb1b42..e6423b03b2af8 100644 --- a/executor/projection.go +++ b/executor/projection.go @@ -172,6 +172,9 @@ func (e *ProjectionExec) unParallelExecute(ctx context.Context, chk *chunk.Chunk if err != nil { return err } + if e.childResult.NumRows() == 0 { + return nil + } err = e.evaluatorSuit.Run(e.ctx, e.childResult, chk) return err } diff --git a/expression/builtin_vectorized_test.go b/expression/builtin_vectorized_test.go index ceb4595d072ab..fd6a08cbc3ec5 100644 --- a/expression/builtin_vectorized_test.go +++ b/expression/builtin_vectorized_test.go @@ -740,3 +740,54 @@ func (s *testEvaluatorSuite) TestVectorizedCheck(c *C) { sf = &ScalarFunction{Function: rowF} c.Assert(sf.Vectorized(), IsFalse) } + +func genFloat32Col() (*Column, *chunk.Chunk, *chunk.Column) { + typeFloat := types.NewFieldType(mysql.TypeFloat) + col := &Column{Index: 0, RetType: typeFloat} + chk := chunk.NewChunkWithCapacity([]*types.FieldType{typeFloat}, 1024) + for i := 0; i < 1024; i++ { + chk.AppendFloat32(0, rand.Float32()) + } + result := chunk.NewColumn(typeFloat, 1024) + return col, chk, result +} + +func (s *testEvaluatorSuite) TestFloat32ColVec(c *C) { + col, chk, result := genFloat32Col() + ctx := mock.NewContext() + c.Assert(col.VecEvalReal(ctx, chk, result), IsNil) + it := chunk.NewIterator4Chunk(chk) + i := 0 + for row := it.Begin(); row != it.End(); row = it.Next() { + v, _, err := col.EvalReal(ctx, row) + c.Assert(err, IsNil) + c.Assert(v, Equals, result.GetFloat64(i)) + i++ + } +} + +func BenchmarkFloat32ColRow(b *testing.B) { + col, chk, _ := genFloat32Col() + ctx := mock.NewContext() + it := chunk.NewIterator4Chunk(chk) + b.ResetTimer() + for i := 0; i < b.N; i++ { + for row := it.Begin(); row != it.End(); row = it.Next() { + if _, _, err := col.EvalReal(ctx, row); err != nil { + b.Fatal(err) + } + + } + } +} + +func BenchmarkFloat32ColVec(b *testing.B) { + col, chk, result := genFloat32Col() + ctx := mock.NewContext() + b.ResetTimer() + for i := 0; i < b.N; i++ { + if err := col.VecEvalReal(ctx, chk, result); err != nil { + b.Fatal(err) + } + } +} diff --git a/expression/chunk_executor.go b/expression/chunk_executor.go index 7294c9d85dd95..7666965081700 100644 --- a/expression/chunk_executor.go +++ b/expression/chunk_executor.go @@ -95,8 +95,9 @@ func evalOneVec(ctx sessionctx.Context, expr Expression, input *chunk.Chunk, out } if ft.Tp == mysql.TypeFloat { f64s := result.Float64s() - buf := chunk.NewColumn(ft, input.NumRows()) - buf.ResizeFloat32(input.NumRows()) + n := input.NumRows() + buf := chunk.NewColumn(ft, n) + buf.ResizeFloat32(n, false) f32s := buf.Float32s() for i := range f64s { if result.IsNull(i) { diff --git a/expression/column.go b/expression/column.go index 055f22207c688..d0f37fbc2b5b8 100644 --- a/expression/column.go +++ b/expression/column.go @@ -240,6 +240,22 @@ func (col *Column) VecEvalInt(ctx sessionctx.Context, input *chunk.Chunk, result // VecEvalReal evaluates this expression in a vectorized manner. func (col *Column) VecEvalReal(ctx sessionctx.Context, input *chunk.Chunk, result *chunk.Column) error { + n := input.NumRows() + src := input.Column(col.Index) + if col.GetType().Tp == mysql.TypeFloat { + result.ResizeFloat64(n, false) + f32s := src.Float32s() + f64s := result.Float64s() + for i := range f32s { + // TODO(zhangyuanjia): speed up the way to manipulate null-bitmaps. + if src.IsNull(i) { + result.SetNull(i, true) + } else { + f64s[i] = float64(f32s[i]) + } + } + return nil + } input.Column(col.Index).CopyReconstruct(input.Sel(), result) return nil } diff --git a/expression/constant.go b/expression/constant.go index 14aea4de282f0..be33d1f17ce41 100644 --- a/expression/constant.go +++ b/expression/constant.go @@ -162,7 +162,7 @@ func (c *Constant) VecEvalTime(ctx sessionctx.Context, input *chunk.Chunk, resul // VecEvalDuration evaluates this expression in a vectorized manner. func (c *Constant) VecEvalDuration(ctx sessionctx.Context, input *chunk.Chunk, result *chunk.Column) error { if c.DeferredExpr == nil { - return genVecFromConstExpr(ctx, c, types.ETDecimal, input, result) + return genVecFromConstExpr(ctx, c, types.ETDuration, input, result) } return c.DeferredExpr.VecEvalDuration(ctx, input, result) } @@ -443,5 +443,8 @@ func (c *Constant) resolveIndices(_ *Schema) error { // Vectorized returns if this expression supports vectorized evaluation. func (c *Constant) Vectorized() bool { + if c.DeferredExpr != nil { + return c.DeferredExpr.Vectorized() + } return true } diff --git a/expression/integration_test.go b/expression/integration_test.go index 79d2352b96752..77ccfe441f199 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -4869,6 +4869,12 @@ func (s *testIntegrationSuite) TestIssue11594(c *C) { tk.MustQuery("SELECT sum(COALESCE(cast(null+rand() as unsigned), v)) FROM t1;").Check(testkit.Rows("3")) } +func (s *testIntegrationSuite) TestDefEnableVectorizedEvaluation(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use mysql") + tk.MustQuery(`select @@tidb_enable_vectorized_expression`).Check(testkit.Rows("1")) +} + func (s *testIntegrationSuite) TestIssue11309And11319(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/expression/vectorized.go b/expression/vectorized.go index 51ecc20c4878a..fa1cf6ad2cb7a 100644 --- a/expression/vectorized.go +++ b/expression/vectorized.go @@ -24,82 +24,81 @@ func genVecFromConstExpr(ctx sessionctx.Context, expr Expression, targetType typ n := 1 if input != nil { n = input.NumRows() + if n == 0 { + result.Reset() + return nil + } } switch targetType { case types.ETInt: - result.ResizeInt64(n) v, isNull, err := expr.EvalInt(ctx, chunk.Row{}) if err != nil { return err } if isNull { - result.SetNulls(0, n, true) + result.ResizeInt64(n, true) return nil } + result.ResizeInt64(n, false) i64s := result.Int64s() for i := range i64s { i64s[i] = v } - result.SetNulls(0, n, false) case types.ETReal: - result.ResizeFloat64(n) v, isNull, err := expr.EvalReal(ctx, chunk.Row{}) if err != nil { return err } if isNull { - result.SetNulls(0, n, true) + result.ResizeFloat64(n, true) return nil } + result.ResizeFloat64(n, false) f64s := result.Float64s() for i := range f64s { f64s[i] = v } - result.SetNulls(0, n, false) case types.ETDecimal: - result.ResizeDecimal(n) v, isNull, err := expr.EvalDecimal(ctx, chunk.Row{}) if err != nil { return err } if isNull { - result.SetNulls(0, n, true) + result.ResizeDecimal(n, true) return nil } + result.ResizeDecimal(n, false) ds := result.Decimals() for i := range ds { ds[i] = *v } - result.SetNulls(0, n, false) case types.ETDatetime, types.ETTimestamp: - result.Reset() v, isNull, err := expr.EvalTime(ctx, chunk.Row{}) if err != nil { return err } if isNull { - for i := 0; i < n; i++ { - result.AppendNull() - } - } else { - for i := 0; i < n; i++ { - result.AppendTime(v) - } + result.ResizeTime(n, true) + return nil + } + result.ResizeTime(n, false) + ts := result.Times() + for i := range ts { + ts[i] = v } case types.ETDuration: - result.ResizeDuration(n) v, isNull, err := expr.EvalDuration(ctx, chunk.Row{}) if err != nil { return err } if isNull { - for i := 0; i < n; i++ { - result.AppendNull() - } - } else { - for i := 0; i < n; i++ { - result.AppendDuration(v) - } + result.ResizeDuration(n, true) + return nil + } + result.ResizeDuration(n, false) + ds := result.GoDurations() + for i := range ds { + ds[i] = v.Duration } case types.ETJson: result.ReserveJSON(n) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index a76ad0a70a701..0d90099f1523c 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -460,6 +460,7 @@ func NewSessionVars() *SessionVars { CorrelationThreshold: DefOptCorrelationThreshold, CorrelationExpFactor: DefOptCorrelationExpFactor, EnableRadixJoin: false, + EnableVectorizedExpression: DefEnableVectorizedExpression, L2CacheSize: cpuid.CPU.Cache.L2, CommandValue: uint32(mysql.ComSleep), TiDBOptJoinReorderThreshold: DefTiDBOptJoinReorderThreshold, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 7c492d9db47fa..df0f2d0d8c881 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -347,7 +347,7 @@ const ( DefTiDBForcePriority = mysql.NoPriority DefTiDBUseRadixJoin = false DefEnableWindowFunction = true - DefEnableVectorizedExpression = false + DefEnableVectorizedExpression = true DefTiDBOptJoinReorderThreshold = 0 DefTiDBDDLSlowOprThreshold = 300 DefTiDBUseFastAnalyze = false diff --git a/util/chunk/column.go b/util/chunk/column.go index aad386207b601..476e4bb332593 100644 --- a/util/chunk/column.go +++ b/util/chunk/column.go @@ -244,7 +244,7 @@ const ( ) // resize resizes the column so that it contains n elements, only valid for fixed-length types. -func (c *Column) resize(n, typeSize int) { +func (c *Column) resize(n, typeSize int, isNull bool) { sizeData := n * typeSize if cap(c.data) >= sizeData { (*reflect.SliceHeader)(unsafe.Pointer(&c.data)).Len = sizeData @@ -252,11 +252,22 @@ func (c *Column) resize(n, typeSize int) { c.data = make([]byte, sizeData) } + newNulls := false sizeNulls := (n + 7) >> 3 if cap(c.nullBitmap) >= sizeNulls { (*reflect.SliceHeader)(unsafe.Pointer(&c.nullBitmap)).Len = sizeNulls } else { c.nullBitmap = make([]byte, sizeNulls) + newNulls = true + } + if !isNull || !newNulls { + var nullVal byte = 0 + if !isNull { + nullVal = 0xFF + } + for i := range c.nullBitmap { + c.nullBitmap[i] = nullVal + } } if cap(c.elemBuf) >= typeSize { @@ -339,38 +350,38 @@ func (c *Column) nullCount() int { } // ResizeInt64 resizes the column so that it contains n int64 elements. -func (c *Column) ResizeInt64(n int) { - c.resize(n, sizeInt64) +func (c *Column) ResizeInt64(n int, isNull bool) { + c.resize(n, sizeInt64, isNull) } // ResizeUint64 resizes the column so that it contains n uint64 elements. -func (c *Column) ResizeUint64(n int) { - c.resize(n, sizeUint64) +func (c *Column) ResizeUint64(n int, isNull bool) { + c.resize(n, sizeUint64, isNull) } // ResizeFloat32 resizes the column so that it contains n float32 elements. -func (c *Column) ResizeFloat32(n int) { - c.resize(n, sizeFloat32) +func (c *Column) ResizeFloat32(n int, isNull bool) { + c.resize(n, sizeFloat32, isNull) } // ResizeFloat64 resizes the column so that it contains n float64 elements. -func (c *Column) ResizeFloat64(n int) { - c.resize(n, sizeFloat64) +func (c *Column) ResizeFloat64(n int, isNull bool) { + c.resize(n, sizeFloat64, isNull) } // ResizeDecimal resizes the column so that it contains n decimal elements. -func (c *Column) ResizeDecimal(n int) { - c.resize(n, sizeMyDecimal) +func (c *Column) ResizeDecimal(n int, isNull bool) { + c.resize(n, sizeMyDecimal, isNull) } // ResizeDuration resizes the column so that it contains n duration elements. -func (c *Column) ResizeDuration(n int) { - c.resize(n, sizeGoDuration) +func (c *Column) ResizeDuration(n int, isNull bool) { + c.resize(n, sizeGoDuration, isNull) } // ResizeTime resizes the column so that it contains n Time elements. -func (c *Column) ResizeTime(n int) { - c.resize(n, sizeTime) +func (c *Column) ResizeTime(n int, isNull bool) { + c.resize(n, sizeTime, isNull) } // ReserveString changes the column capacity to store n string elements and set the length to zero. diff --git a/util/chunk/column_test.go b/util/chunk/column_test.go index 2a105440b067e..75795555a5a62 100644 --- a/util/chunk/column_test.go +++ b/util/chunk/column_test.go @@ -585,7 +585,7 @@ func (s *testChunkSuite) TestReconstructVarLen(c *check.C) { func (s *testChunkSuite) TestPreAllocInt64(c *check.C) { col := NewColumn(types.NewFieldType(mysql.TypeLonglong), 128) - col.ResizeInt64(256) + col.ResizeInt64(256, true) i64s := col.Int64s() c.Assert(len(i64s), check.Equals, 256) for i := 0; i < 256; i++ { @@ -601,7 +601,7 @@ func (s *testChunkSuite) TestPreAllocUint64(c *check.C) { tll := types.NewFieldType(mysql.TypeLonglong) tll.Flag |= mysql.UnsignedFlag col := NewColumn(tll, 128) - col.ResizeUint64(256) + col.ResizeUint64(256, true) u64s := col.Uint64s() c.Assert(len(u64s), check.Equals, 256) for i := 0; i < 256; i++ { @@ -615,7 +615,7 @@ func (s *testChunkSuite) TestPreAllocUint64(c *check.C) { func (s *testChunkSuite) TestPreAllocFloat32(c *check.C) { col := newFixedLenColumn(sizeFloat32, 128) - col.ResizeFloat32(256) + col.ResizeFloat32(256, true) f32s := col.Float32s() c.Assert(len(f32s), check.Equals, 256) for i := 0; i < 256; i++ { @@ -629,7 +629,7 @@ func (s *testChunkSuite) TestPreAllocFloat32(c *check.C) { func (s *testChunkSuite) TestPreAllocFloat64(c *check.C) { col := newFixedLenColumn(sizeFloat64, 128) - col.ResizeFloat64(256) + col.ResizeFloat64(256, true) f64s := col.Float64s() c.Assert(len(f64s), check.Equals, 256) for i := 0; i < 256; i++ { @@ -643,7 +643,7 @@ func (s *testChunkSuite) TestPreAllocFloat64(c *check.C) { func (s *testChunkSuite) TestPreAllocDecimal(c *check.C) { col := newFixedLenColumn(sizeMyDecimal, 128) - col.ResizeDecimal(256) + col.ResizeDecimal(256, true) ds := col.Decimals() c.Assert(len(ds), check.Equals, 256) for i := 0; i < 256; i++ { @@ -656,7 +656,7 @@ func (s *testChunkSuite) TestPreAllocDecimal(c *check.C) { func (s *testChunkSuite) TestPreAllocTime(c *check.C) { col := newFixedLenColumn(sizeTime, 128) - col.ResizeTime(256) + col.ResizeTime(256, true) ds := col.Times() c.Assert(len(ds), check.Equals, 256) for i := 0; i < 256; i++ { @@ -669,7 +669,7 @@ func (s *testChunkSuite) TestPreAllocTime(c *check.C) { func (s *testChunkSuite) TestNull(c *check.C) { col := newFixedLenColumn(sizeFloat64, 32) - col.ResizeFloat64(1024) + col.ResizeFloat64(1024, true) c.Assert(col.nullCount(), check.Equals, 1024) notNulls := make(map[int]struct{}) @@ -684,16 +684,16 @@ func (s *testChunkSuite) TestNull(c *check.C) { c.Assert(col.IsNull(idx), check.Equals, false) } - col.ResizeFloat64(8) + col.ResizeFloat64(8, true) col.SetNulls(0, 8, true) col.SetNull(7, false) c.Assert(col.nullCount(), check.Equals, 7) - col.ResizeFloat64(8) + col.ResizeFloat64(8, true) col.SetNulls(0, 8, true) c.Assert(col.nullCount(), check.Equals, 8) - col.ResizeFloat64(9) + col.ResizeFloat64(9, true) col.SetNulls(0, 9, true) col.SetNull(8, false) c.Assert(col.nullCount(), check.Equals, 8) @@ -701,7 +701,7 @@ func (s *testChunkSuite) TestNull(c *check.C) { func (s *testChunkSuite) TestSetNulls(c *check.C) { col := newFixedLenColumn(sizeFloat64, 32) - col.ResizeFloat64(1024) + col.ResizeFloat64(1024, true) c.Assert(col.nullCount(), check.Equals, 1024) col.SetNulls(0, 1024, false) @@ -732,11 +732,11 @@ func (s *testChunkSuite) TestResizeReserve(c *check.C) { c.Assert(cI64s.length, check.Equals, 0) for i := 0; i < 100; i++ { t := rand.Intn(1024) - cI64s.ResizeInt64(t) + cI64s.ResizeInt64(t, true) c.Assert(cI64s.length, check.Equals, t) c.Assert(len(cI64s.Int64s()), check.Equals, t) } - cI64s.ResizeInt64(0) + cI64s.ResizeInt64(0, true) c.Assert(cI64s.length, check.Equals, 0) c.Assert(len(cI64s.Int64s()), check.Equals, 0) @@ -821,7 +821,7 @@ func BenchmarkDurationVec(b *testing.B) { b.ResetTimer() for k := 0; k < b.N; k++ { - result.ResizeDuration(1024) + result.ResizeDuration(1024, true) for i := 0; i < 1024; i++ { d1 := types.Duration{Duration: ds1[i]} d2 := types.Duration{Duration: ds2[i]} @@ -875,7 +875,7 @@ func BenchmarkTimeVec(b *testing.B) { b.ResetTimer() for k := 0; k < b.N; k++ { - result.ResizeTime(1024) + result.ResizeTime(1024, true) for i := 0; i < 1024; i++ { if r := ds1[i].Compare(ds2[i]); r > 0 { rs[i] = ds1[i] From 06fc28f4f74c9162d0cfd60e1ed324c320b20712 Mon Sep 17 00:00:00 2001 From: lysu Date: Tue, 3 Sep 2019 11:07:34 +0800 Subject: [PATCH 21/21] gomod: update region cache btree version (#11988) --- go.mod | 2 +- go.sum | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/go.mod b/go.mod index 24ebbadf68552..b7f913019c5f6 100644 --- a/go.mod +++ b/go.mod @@ -17,7 +17,7 @@ require ( github.com/go-sql-driver/mysql v0.0.0-20170715192408-3955978caca4 github.com/gogo/protobuf v1.2.0 github.com/golang/protobuf v1.2.0 - github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c + github.com/google/btree v1.0.0 github.com/google/uuid v1.1.1 github.com/gorilla/context v1.1.1 // indirect github.com/gorilla/mux v1.6.2 diff --git a/go.sum b/go.sum index 855f4dd8adb93..c628a3197a912 100644 --- a/go.sum +++ b/go.sum @@ -75,6 +75,8 @@ github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8l github.com/google/btree v0.0.0-20180124185431-e89373fe6b4a/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c h1:964Od4U6p2jUkFxvCydnIczKteheJEzHRToSGK3Bnlw= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/btree v1.0.0 h1:0udJVsspx3VBr5FwtLhQQtuAsVc79tTq0ocGIPAU6qo= +github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/shlex v0.0.0-20181106134648-c34317bd91bf/go.mod h1:RpwtwJQFrIEPstU94h88MWPXP2ektJZ8cZ0YntAmXiE= github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.1 h1:Gkbcsh/GbpXz7lPftLA3P6TYMwjCLYm83jiFQZF/3gY=