From cfb89e436c51727a361362a41e53a8541c121c7b Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 4 Jun 2021 17:42:28 +0800 Subject: [PATCH 01/27] executor: skip TestMppExecution when race is enabled (#24222) (#24234) --- executor/tiflash_test.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/executor/tiflash_test.go b/executor/tiflash_test.go index b49630448c6f0..c09d70cde0926 100644 --- a/executor/tiflash_test.go +++ b/executor/tiflash_test.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/unistore" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" + "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" ) @@ -134,6 +135,9 @@ func (s *tiflashTestSuite) TestReadUnsigedPK(c *C) { } func (s *tiflashTestSuite) TestMppExecution(c *C) { + if israce.RaceEnabled { + c.Skip("skip race test because of long running") + } tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") From 5489cd83931735c5b1d15b8425f135655c55fbc7 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 4 Jun 2021 18:02:28 +0800 Subject: [PATCH 02/27] executor: speed up race test TestInsertReorgDelete (#24208) (#24229) --- executor/insert_test.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/executor/insert_test.go b/executor/insert_test.go index 2e3b5eaa64a0e..6e21a0889ad46 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/execdetails" + "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" ) @@ -246,6 +247,9 @@ func (s *testSuite10) TestPaddingCommonHandle(c *C) { } func (s *testSuite2) TestInsertReorgDelete(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") From 4abe75572606fb1892b7cd07ff5b77861ecd7d49 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 4 Jun 2021 18:24:28 +0800 Subject: [PATCH 03/27] executor: Fix unstable TestTiDBLastTxnInfoCommitMode (#24779) (#25160) --- executor/executor_test.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index bdf859b04ef64..ca557d742f211 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -3056,9 +3056,10 @@ func (s *testSerialSuite) TestTiDBLastTxnInfoCommitMode(c *C) { c.Assert(rows[0][1], Equals, "false") c.Assert(rows[0][2], Equals, "false") - config.UpdateGlobal(func(conf *config.Config) { - conf.TiKVClient.AsyncCommit.SafeWindow = 0 - }) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/invalidMaxCommitTS", "return"), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/invalidMaxCommitTS"), IsNil) + }() tk.MustExec("set @@tidb_enable_async_commit = 1") tk.MustExec("set @@tidb_enable_1pc = 0") From dc40a093a0058b95fc859f60d4b84337dbdff9f6 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Sat, 5 Jun 2021 10:06:27 +0800 Subject: [PATCH 04/27] session: fix a bug when updating duplicate bindings (#25168) (#25177) --- session/bootstrap.go | 8 +++++++- session/bootstrap_test.go | 22 ++++++++++++++++++++-- 2 files changed, 27 insertions(+), 3 deletions(-) diff --git a/session/bootstrap.go b/session/bootstrap.go index c87a79cbbce21..74b4d53d3055f 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -1392,6 +1392,12 @@ func updateBindInfo(iter *chunk.Iterator4Chunk, p *parser.Parser, bindMap map[st for row := iter.Begin(); row != iter.End(); row = iter.Next() { bind := row.GetString(0) db := row.GetString(1) + status := row.GetString(2) + + if status != "using" && status != "builtin" { + continue + } + charset := row.GetString(4) collation := row.GetString(5) stmt, err := p.ParseOneStmt(bind, charset, collation) @@ -1410,7 +1416,7 @@ func updateBindInfo(iter *chunk.Iterator4Chunk, p *parser.Parser, bindMap map[st } bindMap[originWithDB] = bindInfo{ bindSQL: utilparser.RestoreWithDefaultDB(stmt, db, bind), - status: row.GetString(2), + status: status, createTime: row.GetTime(3), charset: charset, collation: collation, diff --git a/session/bootstrap_test.go b/session/bootstrap_test.go index e7e8398845d88..58cc58af06c86 100644 --- a/session/bootstrap_test.go +++ b/session/bootstrap_test.go @@ -591,18 +591,36 @@ func (s *testBootstrapSuite) TestUpdateDuplicateBindInfo(c *C) { // The latest one. mustExecSQL(c, se, `insert into mysql.bind_info values('select * from test . t', 'select /*+ use_index(t, idx_b)*/ * from test.t', 'test', 'using', '2021-01-04 14:50:58.257', '2021-01-09 14:50:58.257', 'utf8', 'utf8_general_ci', 'manual')`) + mustExecSQL(c, se, `insert into mysql.bind_info values('select * from t where a < ?', 'select * from t use index(idx) where a < 1', 'test', 'deleted', '2021-06-04 17:04:43.333', '2021-06-04 17:04:43.335', 'utf8', 'utf8_general_ci', 'manual')`) + mustExecSQL(c, se, `insert into mysql.bind_info values('select * from t where a < ?', 'select * from t ignore index(idx) where a < 1', 'test', 'using', '2021-06-04 17:04:43.335', '2021-06-04 17:04:43.335', 'utf8', 'utf8_general_ci', 'manual')`) + mustExecSQL(c, se, `insert into mysql.bind_info values('select * from test . t where a <= ?', 'select * from test.t use index(idx) where a <= 1', '', 'deleted', '2021-06-04 17:04:43.345', '2021-06-04 17:04:45.334', 'utf8', 'utf8_general_ci', 'manual')`) + mustExecSQL(c, se, `insert into mysql.bind_info values('select * from test . t where a <= ?', 'select * from test.t ignore index(idx) where a <= 1', '', 'using', '2021-06-04 17:04:45.334', '2021-06-04 17:04:45.334', 'utf8', 'utf8_general_ci', 'manual')`) + upgradeToVer67(se, version66) - r := mustExecSQL(c, se, `select original_sql, bind_sql, default_db, status, create_time from mysql.bind_info where source != 'builtin'`) + r := mustExecSQL(c, se, `select original_sql, bind_sql, default_db, status, create_time from mysql.bind_info where source != 'builtin' order by create_time`) req := r.NewChunk() c.Assert(r.Next(ctx, req), IsNil) - c.Assert(req.NumRows(), Equals, 1) + c.Assert(req.NumRows(), Equals, 3) row := req.GetRow(0) c.Assert(row.GetString(0), Equals, "select * from `test` . `t`") c.Assert(row.GetString(1), Equals, "SELECT /*+ use_index(`t` `idx_b`)*/ * FROM `test`.`t`") c.Assert(row.GetString(2), Equals, "") c.Assert(row.GetString(3), Equals, "using") c.Assert(row.GetTime(4).String(), Equals, "2021-01-04 14:50:58.257") + row = req.GetRow(1) + c.Assert(row.GetString(0), Equals, "select * from `test` . `t` where `a` < ?") + c.Assert(row.GetString(1), Equals, "SELECT * FROM `test`.`t` IGNORE INDEX (`idx`) WHERE `a` < 1") + c.Assert(row.GetString(2), Equals, "") + c.Assert(row.GetString(3), Equals, "using") + c.Assert(row.GetTime(4).String(), Equals, "2021-06-04 17:04:43.335") + row = req.GetRow(2) + c.Assert(row.GetString(0), Equals, "select * from `test` . `t` where `a` <= ?") + c.Assert(row.GetString(1), Equals, "SELECT * FROM `test`.`t` IGNORE INDEX (`idx`) WHERE `a` <= 1") + c.Assert(row.GetString(2), Equals, "") + c.Assert(row.GetString(3), Equals, "using") + c.Assert(row.GetTime(4).String(), Equals, "2021-06-04 17:04:45.334") + c.Assert(r.Close(), IsNil) mustExecSQL(c, se, "delete from mysql.bind_info where original_sql = 'select * from test . t'") } From 083d6a7bbbbbe7ee89b9f558960b9dcf271154e1 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Wed, 16 Jun 2021 13:56:37 +0800 Subject: [PATCH 05/27] expression: Maintain separate scalar function pushdown lists for each engine instead of unified. (#23284) (#23661) --- expression/expr_to_pb_test.go | 83 ++++++++- expression/expression.go | 325 +++++++++++----------------------- 2 files changed, 179 insertions(+), 229 deletions(-) diff --git a/expression/expr_to_pb_test.go b/expression/expr_to_pb_test.go index b215fc499cd1f..35cd5dd9ab322 100644 --- a/expression/expr_to_pb_test.go +++ b/expression/expr_to_pb_test.go @@ -613,6 +613,9 @@ func (s *testEvaluatorSuite) TestExprPushDownToFlash(c *C) { decimalColumn := dg.genColumn(mysql.TypeNewDecimal, 4) stringColumn := dg.genColumn(mysql.TypeString, 5) datetimeColumn := dg.genColumn(mysql.TypeDatetime, 6) + binaryStringColumn := dg.genColumn(mysql.TypeString, 7) + binaryStringColumn.RetType.Collate = charset.CollationBin + function, err := NewFunction(mock.NewContext(), ast.JSONLength, types.NewFieldType(mysql.TypeLonglong), jsonColumn) c.Assert(err, IsNil) exprs = append(exprs, function) @@ -724,9 +727,39 @@ func (s *testEvaluatorSuite) TestExprPushDownToFlash(c *C) { c.Assert(err, IsNil) exprs = append(exprs, function) + // Substring2ArgsUTF8 + function, err = NewFunction(mock.NewContext(), ast.Substr, types.NewFieldType(mysql.TypeString), stringColumn, intColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + + // Substring3ArgsUTF8 + function, err = NewFunction(mock.NewContext(), ast.Substr, types.NewFieldType(mysql.TypeString), stringColumn, intColumn, intColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + + function, err = NewFunction(mock.NewContext(), ast.Round, types.NewFieldType(mysql.TypeDouble), realColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + + function, err = NewFunction(mock.NewContext(), ast.Round, types.NewFieldType(mysql.TypeLonglong), intColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + canPush := CanExprsPushDown(sc, exprs, client, kv.TiFlash) c.Assert(canPush, Equals, true) + exprs = exprs[:0] + + // Substring2Args: can not be pushed + function, err = NewFunction(mock.NewContext(), ast.Substr, types.NewFieldType(mysql.TypeString), binaryStringColumn, intColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + + // Substring3Args: can not be pushed + function, err = NewFunction(mock.NewContext(), ast.Substr, types.NewFieldType(mysql.TypeString), binaryStringColumn, intColumn, intColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + function, err = NewFunction(mock.NewContext(), ast.JSONDepth, types.NewFieldType(mysql.TypeLonglong), jsonColumn) c.Assert(err, IsNil) exprs = append(exprs, function) @@ -736,23 +769,57 @@ func (s *testEvaluatorSuite) TestExprPushDownToFlash(c *C) { c.Assert(err, IsNil) exprs = append(exprs, function) + // RoundDecimal: can not be pushed + function, err = NewFunction(mock.NewContext(), ast.Round, types.NewFieldType(mysql.TypeNewDecimal), decimalColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + pushed, remained := PushDownExprs(sc, exprs, client, kv.TiFlash) - c.Assert(len(pushed), Equals, len(exprs)-2) - c.Assert(len(remained), Equals, 2) + c.Assert(len(pushed), Equals, 0) + c.Assert(len(remained), Equals, len(exprs)) } func (s *testEvaluatorSuite) TestExprOnlyPushDownToFlash(c *C) { sc := new(stmtctx.StatementContext) client := new(mock.Client) dg := new(dataGen4Expr2PbTest) - function, err := NewFunction(mock.NewContext(), ast.TimestampDiff, types.NewFieldType(mysql.TypeLonglong), - dg.genColumn(mysql.TypeString, 1), dg.genColumn(mysql.TypeDatetime, 2), dg.genColumn(mysql.TypeDatetime, 3)) + exprs := make([]Expression, 0) + + //jsonColumn := dg.genColumn(mysql.TypeJSON, 1) + intColumn := dg.genColumn(mysql.TypeLonglong, 2) + //realColumn := dg.genColumn(mysql.TypeDouble, 3) + decimalColumn := dg.genColumn(mysql.TypeNewDecimal, 4) + stringColumn := dg.genColumn(mysql.TypeString, 5) + datetimeColumn := dg.genColumn(mysql.TypeDatetime, 6) + binaryStringColumn := dg.genColumn(mysql.TypeString, 7) + binaryStringColumn.RetType.Collate = charset.CollationBin + + function, err := NewFunction(mock.NewContext(), ast.Substr, types.NewFieldType(mysql.TypeString), stringColumn, intColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + + function, err = NewFunction(mock.NewContext(), ast.Substring, types.NewFieldType(mysql.TypeString), stringColumn, intColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + + function, err = NewFunction(mock.NewContext(), ast.DateAdd, types.NewFieldType(mysql.TypeDatetime), datetimeColumn, intColumn, stringColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + + function, err = NewFunction(mock.NewContext(), ast.TimestampDiff, types.NewFieldType(mysql.TypeLonglong), stringColumn, datetimeColumn, datetimeColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + + function, err = NewFunction(mock.NewContext(), ast.FromUnixTime, types.NewFieldType(mysql.TypeDatetime), decimalColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + + function, err = NewFunction(mock.NewContext(), ast.Extract, types.NewFieldType(mysql.TypeLonglong), stringColumn, datetimeColumn) c.Assert(err, IsNil) - var exprs = make([]Expression, 0) exprs = append(exprs, function) pushed, remained := PushDownExprs(sc, exprs, client, kv.UnSpecified) - c.Assert(len(pushed), Equals, 1) + c.Assert(len(pushed), Equals, len(exprs)) c.Assert(len(remained), Equals, 0) canPush := CanExprsPushDown(sc, exprs, client, kv.TiFlash) @@ -761,12 +828,12 @@ func (s *testEvaluatorSuite) TestExprOnlyPushDownToFlash(c *C) { c.Assert(canPush, Equals, false) pushed, remained = PushDownExprs(sc, exprs, client, kv.TiFlash) - c.Assert(len(pushed), Equals, 1) + c.Assert(len(pushed), Equals, len(exprs)) c.Assert(len(remained), Equals, 0) pushed, remained = PushDownExprs(sc, exprs, client, kv.TiKV) c.Assert(len(pushed), Equals, 0) - c.Assert(len(remained), Equals, 1) + c.Assert(len(remained), Equals, len(exprs)) } func (s *testEvaluatorSuite) TestExprOnlyPushDownToTiKV(c *C) { diff --git a/expression/expression.go b/expression/expression.go index 26bf0fb6e18bf..06b560f32d7b4 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -930,202 +930,152 @@ func IsBinaryLiteral(expr Expression) bool { return ok && con.Value.Kind() == types.KindBinaryLiteral } -func canFuncBePushed(sf *ScalarFunction, storeType kv.StoreType) bool { - // Use the failpoint to control whether to push down an expression in the integration test. - // Push down all expression if the `failpoint expression` is `all`, otherwise, check - // whether scalar function's name is contained in the enabled expression list (e.g.`ne,eq,lt`). - // If neither of the above is true, switch to original logic. - failpoint.Inject("PushDownTestSwitcher", func(val failpoint.Value) { - enabled := val.(string) - if enabled == "all" { - failpoint.Return(true) - } - exprs := strings.Split(enabled, ",") - for _, expr := range exprs { - if strings.ToLower(strings.TrimSpace(expr)) == sf.FuncName.L { - failpoint.Return(true) - } - } - }) - - ret := false +func scalarExprSupportedByTiKV(sf *ScalarFunction) bool { switch sf.FuncName.L { case // op functions. - ast.LogicAnd, - ast.LogicOr, - ast.LogicXor, - ast.UnaryNot, - ast.And, - ast.Or, - ast.Xor, - ast.BitNeg, - ast.LeftShift, - ast.RightShift, - ast.UnaryMinus, + ast.LogicAnd, ast.LogicOr, ast.LogicXor, ast.UnaryNot, ast.And, ast.Or, ast.Xor, ast.BitNeg, ast.LeftShift, ast.RightShift, ast.UnaryMinus, // compare functions. - ast.LT, - ast.LE, - ast.EQ, - ast.NE, - ast.GE, - ast.GT, - ast.NullEQ, - ast.In, - ast.IsNull, - ast.Like, - ast.IsTruthWithoutNull, - ast.IsTruthWithNull, - ast.IsFalsity, + ast.LT, ast.LE, ast.EQ, ast.NE, ast.GE, ast.GT, ast.NullEQ, ast.In, ast.IsNull, ast.Like, ast.IsTruthWithoutNull, ast.IsTruthWithNull, ast.IsFalsity, // arithmetical functions. - ast.Plus, - ast.Minus, - ast.Mul, - ast.Div, - ast.Abs, + ast.Plus, ast.Minus, ast.Mul, ast.Div, ast.Abs, /*ast.Mod,*/ // math functions. - ast.Ceil, - ast.Ceiling, - ast.Floor, - ast.Sqrt, - ast.Sign, - ast.Ln, - ast.Log, - ast.Log2, - ast.Log10, - ast.Exp, - ast.Pow, + ast.Ceil, ast.Ceiling, ast.Floor, ast.Sqrt, ast.Sign, ast.Ln, ast.Log, ast.Log2, ast.Log10, ast.Exp, ast.Pow, + // Rust use the llvm math functions, which have different precision with Golang/MySQL(cmath) // open the following switchers if we implement them in coprocessor via `cmath` - // ast.Sin, - // ast.Asin, - // ast.Cos, - // ast.Acos, - // ast.Tan, - // ast.Atan, - // ast.Atan2, - // ast.Cot, - ast.Radians, - ast.Degrees, - ast.Conv, - ast.CRC32, + // ast.Sin, ast.Asin, ast.Cos, ast.Acos, ast.Tan, ast.Atan, ast.Atan2, ast.Cot, + ast.Radians, ast.Degrees, ast.Conv, ast.CRC32, // control flow functions. - ast.Case, - ast.If, - ast.Ifnull, - ast.Coalesce, + ast.Case, ast.If, ast.Ifnull, ast.Coalesce, // string functions. - ast.Length, - ast.BitLength, - ast.Concat, - ast.ConcatWS, - // ast.Locate, - ast.Replace, - ast.ASCII, - ast.Hex, - ast.Reverse, - ast.LTrim, - ast.RTrim, - // ast.Left, - ast.Strcmp, - ast.Space, - ast.Elt, - ast.Field, + ast.Length, ast.BitLength, ast.Concat, ast.ConcatWS /*ast.Locate,*/, ast.Replace, ast.ASCII, ast.Hex, + ast.Reverse, ast.LTrim, ast.RTrim /*ast.Left,*/, ast.Strcmp, ast.Space, ast.Elt, ast.Field, // json functions. - ast.JSONType, - ast.JSONExtract, + ast.JSONType, ast.JSONExtract, ast.JSONObject, ast.JSONArray, ast.JSONMerge, ast.JSONSet, + ast.JSONInsert /*ast.JSONReplace,*/, ast.JSONRemove, ast.JSONLength, // FIXME: JSONUnquote is incompatible with Coprocessor // ast.JSONUnquote, - ast.JSONObject, - ast.JSONArray, - ast.JSONMerge, - ast.JSONSet, - ast.JSONInsert, - // ast.JSONReplace, - ast.JSONRemove, - ast.JSONLength, // date functions. - ast.DateFormat, - ast.FromDays, - // ast.ToDays, - ast.DayOfYear, - ast.DayOfMonth, - ast.Year, - ast.Month, + ast.DateFormat, ast.FromDays /*ast.ToDays,*/, ast.DayOfYear, ast.DayOfMonth, ast.Year, ast.Month, // FIXME: the coprocessor cannot keep the same behavior with TiDB in current compute framework - // ast.Hour, - // ast.Minute, - // ast.Second, - // ast.MicroSecond, - // ast.DayName, - ast.PeriodAdd, - ast.PeriodDiff, - ast.TimestampDiff, - ast.DateAdd, - ast.FromUnixTime, - ast.Extract, + // ast.Hour, ast.Minute, ast.Second, ast.MicroSecond, ast.DayName, + ast.PeriodAdd, ast.PeriodDiff, /*ast.TimestampDiff, ast.DateAdd, ast.FromUnixTime,*/ // encryption functions. - ast.MD5, - ast.SHA1, - ast.UncompressedLength, + ast.MD5, ast.SHA1, ast.UncompressedLength, ast.Cast, // misc functions. - ast.InetNtoa, - ast.InetAton, - ast.Inet6Ntoa, - ast.Inet6Aton, - ast.IsIPv4, - ast.IsIPv4Compat, - ast.IsIPv4Mapped, - ast.IsIPv6, - ast.UUID: - ret = true + ast.InetNtoa, ast.InetAton, ast.Inet6Ntoa, ast.Inet6Aton, ast.IsIPv4, ast.IsIPv4Compat, ast.IsIPv4Mapped, ast.IsIPv6, ast.UUID: + + return true // A special case: Only push down Round by signature case ast.Round: switch sf.Function.PbCode() { - case - tipb.ScalarFuncSig_RoundReal, - tipb.ScalarFuncSig_RoundInt, - tipb.ScalarFuncSig_RoundDec: - ret = true + case tipb.ScalarFuncSig_RoundReal, tipb.ScalarFuncSig_RoundInt, tipb.ScalarFuncSig_RoundDec: + return true } - case - ast.Substring, - ast.Substr: + case ast.Rand: switch sf.Function.PbCode() { + case tipb.ScalarFuncSig_RandWithSeedFirstGen: + return true + } + } + return false +} + +func scalarExprSupportedByFlash(function *ScalarFunction) bool { + switch function.FuncName.L { + case + ast.LogicOr, ast.LogicAnd, ast.UnaryNot, ast.BitNeg, ast.Xor, ast.And, ast.Or, + ast.GE, ast.LE, ast.EQ, ast.NE, ast.LT, ast.GT, ast.In, ast.IsNull, ast.Like, + ast.Plus, ast.Minus, ast.Div, ast.Mul, /*ast.Mod,*/ + ast.If, ast.Ifnull, ast.Case, + ast.Month, + ast.TimestampDiff, ast.DateFormat, ast.FromUnixTime, + ast.JSONLength: + return true + case ast.Substr, ast.Substring: + switch function.Function.PbCode() { case tipb.ScalarFuncSig_Substring2ArgsUTF8, tipb.ScalarFuncSig_Substring3ArgsUTF8: - ret = true + return true } - case ast.Rand: - switch sf.Function.PbCode() { - case - tipb.ScalarFuncSig_RandWithSeedFirstGen: - ret = true + case ast.Cast: + switch function.Function.PbCode() { + case tipb.ScalarFuncSig_CastIntAsInt, tipb.ScalarFuncSig_CastIntAsDecimal, tipb.ScalarFuncSig_CastIntAsString, tipb.ScalarFuncSig_CastIntAsTime, + tipb.ScalarFuncSig_CastRealAsInt, tipb.ScalarFuncSig_CastRealAsDecimal, tipb.ScalarFuncSig_CastRealAsString, tipb.ScalarFuncSig_CastRealAsTime, + tipb.ScalarFuncSig_CastStringAsInt, tipb.ScalarFuncSig_CastStringAsDecimal, tipb.ScalarFuncSig_CastStringAsString, tipb.ScalarFuncSig_CastStringAsTime, + tipb.ScalarFuncSig_CastDecimalAsInt, tipb.ScalarFuncSig_CastDecimalAsDecimal, tipb.ScalarFuncSig_CastDecimalAsString, tipb.ScalarFuncSig_CastDecimalAsTime, + tipb.ScalarFuncSig_CastTimeAsInt, tipb.ScalarFuncSig_CastTimeAsDecimal, tipb.ScalarFuncSig_CastTimeAsTime: + return true + } + case ast.DateAdd: + switch function.Function.PbCode() { + case tipb.ScalarFuncSig_AddDateDatetimeInt, tipb.ScalarFuncSig_AddDateStringInt: + return true + } + case ast.Round: + switch function.Function.PbCode() { + case tipb.ScalarFuncSig_RoundInt, tipb.ScalarFuncSig_RoundReal: + return true + } + case ast.Extract: + switch function.Function.PbCode() { + case tipb.ScalarFuncSig_ExtractDatetime: + return true } } - if ret { - switch storeType { - case kv.TiFlash: - ret = scalarExprSupportedByFlash(sf) - case kv.TiKV: - ret = scalarExprSupportedByTiKV(sf) - case kv.TiDB: - ret = scalarExprSupportedByTiDB(sf) + return false +} + +func scalarExprSupportedByTiDB(function *ScalarFunction) bool { + // TiDB can support all functions, but TiPB may not include some functions. + return scalarExprSupportedByTiKV(function) || scalarExprSupportedByFlash(function) +} + +func canFuncBePushed(sf *ScalarFunction, storeType kv.StoreType) bool { + // Use the failpoint to control whether to push down an expression in the integration test. + // Push down all expression if the `failpoint expression` is `all`, otherwise, check + // whether scalar function's name is contained in the enabled expression list (e.g.`ne,eq,lt`). + // If neither of the above is true, switch to original logic. + failpoint.Inject("PushDownTestSwitcher", func(val failpoint.Value) { + enabled := val.(string) + if enabled == "all" { + failpoint.Return(true) } + exprs := strings.Split(enabled, ",") + for _, expr := range exprs { + if strings.ToLower(strings.TrimSpace(expr)) == sf.FuncName.L { + failpoint.Return(true) + } + } + }) + + ret := false + + switch storeType { + case kv.TiFlash: + ret = scalarExprSupportedByFlash(sf) + case kv.TiKV: + ret = scalarExprSupportedByTiKV(sf) + case kv.TiDB: + ret = scalarExprSupportedByTiDB(sf) + case kv.UnSpecified: + ret = scalarExprSupportedByTiDB(sf) || scalarExprSupportedByTiKV(sf) || scalarExprSupportedByFlash(sf) } + if ret { ret = IsPushDownEnabled(sf.FuncName.L, storeType) } @@ -1241,73 +1191,6 @@ func CanExprsPushDown(sc *stmtctx.StatementContext, exprs []Expression, client k return len(remained) == 0 } -func scalarExprSupportedByTiKV(function *ScalarFunction) bool { - switch function.FuncName.L { - case ast.Substr, ast.Substring, ast.DateAdd, ast.TimestampDiff, - ast.FromUnixTime, ast.Extract: - return false - default: - return true - } -} - -func scalarExprSupportedByTiDB(function *ScalarFunction) bool { - switch function.FuncName.L { - case ast.Substr, ast.Substring, ast.DateAdd, ast.TimestampDiff, - ast.FromUnixTime: - return false - default: - return true - } -} - -func scalarExprSupportedByFlash(function *ScalarFunction) bool { - switch function.FuncName.L { - case ast.Plus, ast.Minus, ast.Div, ast.Mul, ast.GE, ast.LE, - ast.EQ, ast.NE, ast.LT, ast.GT, ast.Ifnull, ast.IsNull, - ast.Or, ast.In, ast.Mod, ast.And, ast.LogicOr, ast.LogicAnd, - ast.Like, ast.UnaryNot, ast.Case, ast.Month, ast.Substr, - ast.Substring, ast.TimestampDiff, ast.DateFormat, ast.FromUnixTime, - ast.JSONLength, ast.If, ast.BitNeg, ast.Xor: - return true - case ast.Cast: - switch function.Function.PbCode() { - case tipb.ScalarFuncSig_CastIntAsInt, tipb.ScalarFuncSig_CastIntAsDecimal, tipb.ScalarFuncSig_CastIntAsString, tipb.ScalarFuncSig_CastIntAsTime, - tipb.ScalarFuncSig_CastRealAsInt, tipb.ScalarFuncSig_CastRealAsDecimal, tipb.ScalarFuncSig_CastRealAsString, tipb.ScalarFuncSig_CastRealAsTime, - tipb.ScalarFuncSig_CastStringAsInt, tipb.ScalarFuncSig_CastStringAsDecimal, tipb.ScalarFuncSig_CastStringAsString, tipb.ScalarFuncSig_CastStringAsTime, - tipb.ScalarFuncSig_CastDecimalAsInt, tipb.ScalarFuncSig_CastDecimalAsDecimal, tipb.ScalarFuncSig_CastDecimalAsString, tipb.ScalarFuncSig_CastDecimalAsTime, - tipb.ScalarFuncSig_CastTimeAsInt, tipb.ScalarFuncSig_CastTimeAsDecimal, tipb.ScalarFuncSig_CastTimeAsTime: - return true - default: - return false - } - case ast.DateAdd: - switch function.Function.PbCode() { - case tipb.ScalarFuncSig_AddDateDatetimeInt, tipb.ScalarFuncSig_AddDateStringInt: - return true - default: - return false - } - case ast.Round: - switch function.Function.PbCode() { - case tipb.ScalarFuncSig_RoundInt, tipb.ScalarFuncSig_RoundReal, - tipb.ScalarFuncSig_RoundDec: - return true - default: - return false - } - case ast.Extract: - switch function.Function.PbCode() { - case tipb.ScalarFuncSig_ExtractDatetime: - return true - default: - return false - } - default: - return false - } -} - // wrapWithIsTrue wraps `arg` with istrue function if the return type of expr is not // type int, otherwise, returns `arg` directly. // The `keepNull` controls what the istrue function will return when `arg` is null: From 943d82c3f9cd0baeccfa09e8eda69287570b9357 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 17 Jun 2021 14:26:37 +0800 Subject: [PATCH 06/27] store/copr: balance region for batch cop task (#24521) (#24724) --- store/copr/batch_coprocessor.go | 201 +++++++++++++++++-- store/copr/mpp.go | 14 +- store/{copr => tikv}/batch_request_sender.go | 51 +++-- store/tikv/region_cache.go | 101 ++++++++++ 4 files changed, 320 insertions(+), 47 deletions(-) rename store/{copr => tikv}/batch_request_sender.go (53%) diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index fa2dc90e0d9d6..04f30bc0e8a88 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -16,6 +16,8 @@ package copr import ( "context" "io" + "math" + "strconv" "sync" "sync/atomic" "time" @@ -24,6 +26,7 @@ import ( "github.com/pingcap/kvproto/pkg/coprocessor" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/log" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/logutil" @@ -38,8 +41,9 @@ import ( type batchCopTask struct { storeAddr string cmdType tikvrpc.CmdType + ctx *tikv.RPCContext - copTasks []copTaskAndRPCContext + regionInfos []tikv.RegionInfo } type batchCopResponse struct { @@ -91,9 +95,152 @@ func (rs *batchCopResponse) RespTime() time.Duration { return rs.respTime } -type copTaskAndRPCContext struct { - task *copTask - ctx *tikv.RPCContext +// balanceBatchCopTask balance the regions between available stores, the basic rule is +// 1. the first region of each original batch cop task belongs to its original store because some +// meta data(like the rpc context) in batchCopTask is related to it +// 2. for the remaining regions: +// if there is only 1 available store, then put the region to the related store +// otherwise, use a greedy algorithm to put it into the store with highest weight +func balanceBatchCopTask(originalTasks []*batchCopTask) []*batchCopTask { + if len(originalTasks) <= 1 { + return originalTasks + } + storeTaskMap := make(map[uint64]*batchCopTask) + storeCandidateRegionMap := make(map[uint64]map[string]tikv.RegionInfo) + totalRegionCandidateNum := 0 + totalRemainingRegionNum := 0 + + for _, task := range originalTasks { + taskStoreID := task.regionInfos[0].AllStores[0] + batchTask := &batchCopTask{ + storeAddr: task.storeAddr, + cmdType: task.cmdType, + ctx: task.ctx, + regionInfos: []tikv.RegionInfo{task.regionInfos[0]}, + } + storeTaskMap[taskStoreID] = batchTask + } + + for _, task := range originalTasks { + taskStoreID := task.regionInfos[0].AllStores[0] + for index, ri := range task.regionInfos { + // for each region, figure out the valid store num + validStoreNum := 0 + if index == 0 { + continue + } + if len(ri.AllStores) <= 1 { + validStoreNum = 1 + } else { + for _, storeID := range ri.AllStores { + if _, ok := storeTaskMap[storeID]; ok { + validStoreNum++ + } + } + } + if validStoreNum == 1 { + // if only one store is valid, just put it to storeTaskMap + storeTaskMap[taskStoreID].regionInfos = append(storeTaskMap[taskStoreID].regionInfos, ri) + } else { + // if more than one store is valid, put the region + // to store candidate map + totalRegionCandidateNum += validStoreNum + totalRemainingRegionNum += 1 + taskKey := ri.Region.String() + for _, storeID := range ri.AllStores { + if _, validStore := storeTaskMap[storeID]; !validStore { + continue + } + if _, ok := storeCandidateRegionMap[storeID]; !ok { + candidateMap := make(map[string]tikv.RegionInfo) + storeCandidateRegionMap[storeID] = candidateMap + } + if _, duplicateRegion := storeCandidateRegionMap[storeID][taskKey]; duplicateRegion { + // duplicated region, should not happen, just give up balance + logutil.BgLogger().Warn("Meet duplicated region info during when trying to balance batch cop task, give up balancing") + return originalTasks + } + storeCandidateRegionMap[storeID][taskKey] = ri + } + } + } + } + if totalRemainingRegionNum == 0 { + return originalTasks + } + + avgStorePerRegion := float64(totalRegionCandidateNum) / float64(totalRemainingRegionNum) + findNextStore := func(candidateStores []uint64) uint64 { + store := uint64(math.MaxUint64) + weightedRegionNum := math.MaxFloat64 + if candidateStores != nil { + for _, storeID := range candidateStores { + if _, validStore := storeCandidateRegionMap[storeID]; !validStore { + continue + } + num := float64(len(storeCandidateRegionMap[storeID]))/avgStorePerRegion + float64(len(storeTaskMap[storeID].regionInfos)) + if num < weightedRegionNum { + store = storeID + weightedRegionNum = num + } + } + if store != uint64(math.MaxUint64) { + return store + } + } + for storeID := range storeTaskMap { + if _, validStore := storeCandidateRegionMap[storeID]; !validStore { + continue + } + num := float64(len(storeCandidateRegionMap[storeID]))/avgStorePerRegion + float64(len(storeTaskMap[storeID].regionInfos)) + if num < weightedRegionNum { + store = storeID + weightedRegionNum = num + } + } + return store + } + + store := findNextStore(nil) + for totalRemainingRegionNum > 0 { + if store == uint64(math.MaxUint64) { + break + } + var key string + var ri tikv.RegionInfo + for key, ri = range storeCandidateRegionMap[store] { + // get the first region + break + } + storeTaskMap[store].regionInfos = append(storeTaskMap[store].regionInfos, ri) + totalRemainingRegionNum-- + for _, id := range ri.AllStores { + if _, ok := storeCandidateRegionMap[id]; ok { + delete(storeCandidateRegionMap[id], key) + totalRegionCandidateNum-- + if len(storeCandidateRegionMap[id]) == 0 { + delete(storeCandidateRegionMap, id) + } + } + } + if totalRemainingRegionNum > 0 { + avgStorePerRegion = float64(totalRegionCandidateNum) / float64(totalRemainingRegionNum) + // it is not optimal because we only check the stores that affected by this region, in fact in order + // to find out the store with the lowest weightedRegionNum, all stores should be checked, but I think + // check only the affected stores is more simple and will get a good enough result + store = findNextStore(ri.AllStores) + } + } + if totalRemainingRegionNum > 0 { + logutil.BgLogger().Warn("Some regions are not used when trying to balance batch cop task, give up balancing") + return originalTasks + } + + var ret []*batchCopTask + for _, task := range storeTaskMap { + ret = append(ret, task) + } + return ret } func buildBatchCopTasks(bo *tikv.Backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRanges, storeType kv.StoreType) ([]*batchCopTask, error) { @@ -134,13 +281,15 @@ func buildBatchCopTasks(bo *tikv.Backoffer, cache *tikv.RegionCache, ranges *tik // Then `splitRegion` will reloads these regions. continue } + allStores := cache.GetAllValidTiFlashStores(task.region, rpcCtx.Store) if batchCop, ok := storeTaskMap[rpcCtx.Addr]; ok { - batchCop.copTasks = append(batchCop.copTasks, copTaskAndRPCContext{task: task, ctx: rpcCtx}) + batchCop.regionInfos = append(batchCop.regionInfos, tikv.RegionInfo{Region: task.region, Meta: rpcCtx.Meta, Ranges: task.ranges, AllStores: allStores}) } else { batchTask := &batchCopTask{ - storeAddr: rpcCtx.Addr, - cmdType: cmdType, - copTasks: []copTaskAndRPCContext{{task, rpcCtx}}, + storeAddr: rpcCtx.Addr, + cmdType: cmdType, + ctx: rpcCtx, + regionInfos: []tikv.RegionInfo{{Region: task.region, Meta: rpcCtx.Meta, Ranges: task.ranges, AllStores: allStores}}, } storeTaskMap[rpcCtx.Addr] = batchTask } @@ -153,9 +302,25 @@ func buildBatchCopTasks(bo *tikv.Backoffer, cache *tikv.RegionCache, ranges *tik } continue } + for _, task := range storeTaskMap { batchTasks = append(batchTasks, task) } + if log.GetLevel() <= zap.DebugLevel { + msg := "Before region balance:" + for _, task := range batchTasks { + msg += " store " + task.storeAddr + ": " + strconv.Itoa(len(task.regionInfos)) + " regions," + } + logutil.BgLogger().Debug(msg) + } + batchTasks = balanceBatchCopTask(batchTasks) + if log.GetLevel() <= zap.DebugLevel { + msg := "After region balance:" + for _, task := range batchTasks { + msg += " store " + task.storeAddr + ": " + strconv.Itoa(len(task.regionInfos)) + " regions," + } + logutil.BgLogger().Debug(msg) + } if elapsed := time.Since(start); elapsed > time.Millisecond*500 { logutil.BgLogger().Warn("buildBatchCopTasks takes too much time", @@ -310,8 +475,8 @@ func (b *batchCopIterator) handleTask(ctx context.Context, bo *tikv.Backoffer, t // Merge all ranges and request again. func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *tikv.Backoffer, batchTask *batchCopTask) ([]*batchCopTask, error) { var ranges []kv.KeyRange - for _, taskCtx := range batchTask.copTasks { - taskCtx.task.ranges.Do(func(ran *kv.KeyRange) { + for _, ri := range batchTask.regionInfos { + ri.Ranges.Do(func(ran *kv.KeyRange) { ranges = append(ranges, *ran) }) } @@ -319,16 +484,16 @@ func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *tikv.Backo } func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *tikv.Backoffer, task *batchCopTask) ([]*batchCopTask, error) { - sender := NewRegionBatchRequestSender(b.store.GetRegionCache(), b.store.GetTiKVClient()) + sender := tikv.NewRegionBatchRequestSender(b.store.GetRegionCache(), b.store.GetTiKVClient()) var regionInfos []*coprocessor.RegionInfo - for _, task := range task.copTasks { + for _, ri := range task.regionInfos { regionInfos = append(regionInfos, &coprocessor.RegionInfo{ - RegionId: task.task.region.GetID(), + RegionId: ri.Region.GetID(), RegionEpoch: &metapb.RegionEpoch{ - ConfVer: task.task.region.GetConfVer(), - Version: task.task.region.GetVer(), + ConfVer: ri.Region.GetConfVer(), + Version: ri.Region.GetVer(), }, - Ranges: task.task.ranges.ToPBRanges(), + Ranges: ri.Ranges.ToPBRanges(), }) } @@ -350,8 +515,8 @@ func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *tikv.Backoffe }) req.StoreTp = kv.TiFlash - logutil.BgLogger().Debug("send batch request to ", zap.String("req info", req.String()), zap.Int("cop task len", len(task.copTasks))) - resp, retry, cancel, err := sender.sendStreamReqToAddr(bo, task.copTasks, req, tikv.ReadTimeoutUltraLong) + logutil.BgLogger().Debug("send batch request to ", zap.String("req info", req.String()), zap.Int("cop task len", len(task.regionInfos))) + resp, retry, cancel, err := sender.SendReqToAddr(bo, task.ctx, task.regionInfos, req, tikv.ReadTimeoutUltraLong) // If there are store errors, we should retry for all regions. if retry { return b.retryBatchCopTask(ctx, bo, task) diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 13488ae5b3e03..88ac883bbfb40 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -181,14 +181,14 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *tikv.Backoffer, var regionInfos []*coprocessor.RegionInfo originalTask, ok := req.Meta.(*batchCopTask) if ok { - for _, task := range originalTask.copTasks { + for _, ri := range originalTask.regionInfos { regionInfos = append(regionInfos, &coprocessor.RegionInfo{ - RegionId: task.task.region.GetID(), + RegionId: ri.Region.GetID(), RegionEpoch: &metapb.RegionEpoch{ - ConfVer: task.task.region.GetConfVer(), - Version: task.task.region.GetVer(), + ConfVer: ri.Region.GetConfVer(), + Version: ri.Region.GetVer(), }, - Ranges: task.task.ranges.ToPBRanges(), + Ranges: ri.Ranges.ToPBRanges(), }) } } @@ -215,8 +215,8 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *tikv.Backoffer, // Or else it's the task without region, which always happens in high layer task without table. // In that case if originalTask != nil { - sender := NewRegionBatchRequestSender(m.store.GetRegionCache(), m.store.GetTiKVClient()) - rpcResp, _, _, err = sender.sendStreamReqToAddr(bo, originalTask.copTasks, wrappedReq, tikv.ReadTimeoutMedium) + sender := tikv.NewRegionBatchRequestSender(m.store.GetRegionCache(), m.store.GetTiKVClient()) + rpcResp, _, _, err = sender.SendReqToAddr(bo, originalTask.ctx, originalTask.regionInfos, wrappedReq, tikv.ReadTimeoutMedium) // No matter what the rpc error is, we won't retry the mpp dispatch tasks. // TODO: If we want to retry, we must redo the plan fragment cutting and task scheduling. // That's a hard job but we can try it in the future. diff --git a/store/copr/batch_request_sender.go b/store/tikv/batch_request_sender.go similarity index 53% rename from store/copr/batch_request_sender.go rename to store/tikv/batch_request_sender.go index 7865380f56ee5..cd73efb0430b6 100644 --- a/store/copr/batch_request_sender.go +++ b/store/tikv/batch_request_sender.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package copr +package tikv import ( "context" @@ -19,44 +19,51 @@ import ( "time" "github.com/pingcap/errors" - "github.com/pingcap/tidb/store/tikv" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/store/tikv/tikvrpc" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" ) +// RegionInfo contains region related information for batchCopTask +type RegionInfo struct { + Region RegionVerID + Meta *metapb.Region + Ranges *KeyRanges + AllStores []uint64 +} + // RegionBatchRequestSender sends BatchCop requests to TiFlash server by stream way. type RegionBatchRequestSender struct { - *tikv.RegionRequestSender + *RegionRequestSender } // NewRegionBatchRequestSender creates a RegionBatchRequestSender object. -func NewRegionBatchRequestSender(cache *tikv.RegionCache, client tikv.Client) *RegionBatchRequestSender { +func NewRegionBatchRequestSender(cache *RegionCache, client Client) *RegionBatchRequestSender { return &RegionBatchRequestSender{ - RegionRequestSender: tikv.NewRegionRequestSender(cache, client), + RegionRequestSender: NewRegionRequestSender(cache, client), } } -func (ss *RegionBatchRequestSender) sendStreamReqToAddr(bo *tikv.Backoffer, ctxs []copTaskAndRPCContext, req *tikvrpc.Request, timout time.Duration) (resp *tikvrpc.Response, retry bool, cancel func(), err error) { - // use the first ctx to send request, because every ctx has same address. +// SendReqToAddr send batch cop request +func (ss *RegionBatchRequestSender) SendReqToAddr(bo *Backoffer, rpcCtx *RPCContext, regionInfos []RegionInfo, req *tikvrpc.Request, timout time.Duration) (resp *tikvrpc.Response, retry bool, cancel func(), err error) { cancel = func() {} - rpcCtx := ctxs[0].ctx if e := tikvrpc.SetContext(req, rpcCtx.Meta, rpcCtx.Peer); e != nil { return nil, false, cancel, errors.Trace(e) } ctx := bo.GetCtx() - if rawHook := ctx.Value(tikv.RPCCancellerCtxKey{}); rawHook != nil { - ctx, cancel = rawHook.(*tikv.RPCCanceller).WithCancel(ctx) + if rawHook := ctx.Value(RPCCancellerCtxKey{}); rawHook != nil { + ctx, cancel = rawHook.(*RPCCanceller).WithCancel(ctx) } start := time.Now() resp, err = ss.GetClient().SendRequest(ctx, rpcCtx.Addr, req, timout) if ss.Stats != nil { - tikv.RecordRegionRequestRuntimeStats(ss.Stats, req.Type, time.Since(start)) + RecordRegionRequestRuntimeStats(ss.Stats, req.Type, time.Since(start)) } if err != nil { cancel() ss.SetRPCError(err) - e := ss.onSendFail(bo, ctxs, err) + e := ss.onSendFailForBatchRegions(bo, rpcCtx, regionInfos, err) if e != nil { return nil, false, func() {}, errors.Trace(e) } @@ -66,25 +73,25 @@ func (ss *RegionBatchRequestSender) sendStreamReqToAddr(bo *tikv.Backoffer, ctxs return } -func (ss *RegionBatchRequestSender) onSendFail(bo *tikv.Backoffer, ctxs []copTaskAndRPCContext, err error) error { +func (ss *RegionBatchRequestSender) onSendFailForBatchRegions(bo *Backoffer, ctx *RPCContext, regionInfos []RegionInfo, err error) error { // If it failed because the context is cancelled by ourself, don't retry. if errors.Cause(err) == context.Canceled || status.Code(errors.Cause(err)) == codes.Canceled { return errors.Trace(err) - } else if atomic.LoadUint32(&tikv.ShuttingDown) > 0 { - return tikv.ErrTiDBShuttingDown + } else if atomic.LoadUint32(&ShuttingDown) > 0 { + return ErrTiDBShuttingDown } - for _, failedCtx := range ctxs { - ctx := failedCtx.ctx - if ctx.Meta != nil { - ss.GetRegionCache().OnSendFail(bo, ctx, ss.NeedReloadRegion(ctx), err) - } - } + // The reload region param is always true. Because that every time we try, we must + // re-build the range then re-create the batch sender. As a result, the len of "failStores" + // will change. If tiflash's replica is more than two, the "reload region" will always be false. + // Now that the batch cop and mpp has a relative low qps, it's reasonable to reload every time + // when meeting io error. + ss.GetRegionCache().OnSendFailForBatchRegions(bo, ctx.Store, regionInfos, true, err) // Retry on send request failure when it's not canceled. // When a store is not available, the leader of related region should be elected quickly. // TODO: the number of retry time should be limited:since region may be unavailable // when some unrecoverable disaster happened. - err = bo.Backoff(tikv.BoTiFlashRPC, errors.Errorf("send tikv request error: %v, ctxs: %v, try next peer later", err, ctxs)) + err = bo.Backoff(BoTiFlashRPC, errors.Errorf("send request error: %v, ctx: %v, regionInfos: %v", err, ctx, regionInfos)) return errors.Trace(err) } diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index af6d91ccae03e..ca0b47f1e5cd0 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -110,6 +110,15 @@ func (r *RegionStore) accessStore(mode AccessMode, idx AccessIndex) (int, *Store return sidx, r.stores[sidx] } +func (r *RegionStore) getAccessIndex(mode AccessMode, store *Store) AccessIndex { + for index, sidx := range r.accessIndex[mode] { + if r.stores[sidx].storeID == store.storeID { + return AccessIndex(index) + } + } + return -1 +} + func (r *RegionStore) accessStoreNum(mode AccessMode) int { return len(r.accessIndex[mode]) } @@ -527,6 +536,40 @@ func (c *RegionCache) GetTiKVRPCContext(bo *Backoffer, id RegionVerID, replicaRe }, nil } +// GetAllValidTiFlashStores returns the store ids of all valid TiFlash stores, the store id of currentStore is always the first one +func (c *RegionCache) GetAllValidTiFlashStores(id RegionVerID, currentStore *Store) []uint64 { + // set the cap to 2 because usually, TiFlash table will have 2 replicas + allStores := make([]uint64, 0, 2) + // make sure currentStore id is always the first in allStores + allStores = append(allStores, currentStore.storeID) + ts := time.Now().Unix() + cachedRegion := c.getCachedRegionWithRLock(id) + if cachedRegion == nil { + return allStores + } + if !cachedRegion.checkRegionCacheTTL(ts) { + return allStores + } + regionStore := cachedRegion.getStore() + currentIndex := regionStore.getAccessIndex(TiFlashOnly, currentStore) + if currentIndex == -1 { + return allStores + } + for startOffset := 1; startOffset < regionStore.accessStoreNum(TiFlashOnly); startOffset++ { + accessIdx := AccessIndex((int(currentIndex) + startOffset) % regionStore.accessStoreNum(TiFlashOnly)) + storeIdx, store := regionStore.accessStore(TiFlashOnly, accessIdx) + if store.getResolveState() == needCheck { + continue + } + storeFailEpoch := atomic.LoadUint32(&store.epoch) + if storeFailEpoch != regionStore.storeEpochs[storeIdx] { + continue + } + allStores = append(allStores, store.storeID) + } + return allStores +} + // GetTiFlashRPCContext returns RPCContext for a region must access flash store. If it returns nil, the region // must be out of date and already dropped from cache or not flash store found. // `loadBalance` is an option. For MPP and batch cop, it is pointless and might cause try the failed store repeatly. @@ -664,6 +707,64 @@ func (c *RegionCache) findRegionByKey(bo *Backoffer, key []byte, isEndKey bool) return r, nil } +// OnSendFailForBatchRegions handles send request fail logic. +func (c *RegionCache) OnSendFailForBatchRegions(bo *Backoffer, store *Store, regionInfos []RegionInfo, scheduleReload bool, err error) { + metrics.RegionCacheCounterWithSendFail.Add(float64(len(regionInfos))) + if store.storeType != TiFlash { + logutil.Logger(bo.GetCtx()).Warn("Should not reach here, OnSendFailForBatchRegions only support TiFlash") + return + } + for _, ri := range regionInfos { + if ri.Meta == nil { + continue + } + r := c.getCachedRegionWithRLock(ri.Region) + if r != nil { + peersNum := len(r.meta.Peers) + if len(ri.Meta.Peers) != peersNum { + logutil.Logger(bo.GetCtx()).Info("retry and refresh current region after send request fail and up/down stores length changed", + zap.Stringer("region", &ri.Region), + zap.Bool("needReload", scheduleReload), + zap.Reflect("oldPeers", ri.Meta.Peers), + zap.Reflect("newPeers", r.meta.Peers), + zap.Error(err)) + continue + } + + rs := r.getStore() + + accessMode := TiFlashOnly + accessIdx := rs.getAccessIndex(accessMode, store) + if accessIdx == -1 { + logutil.Logger(bo.GetCtx()).Warn("can not get access index for region " + ri.Region.String()) + continue + } + if err != nil { + storeIdx, s := rs.accessStore(accessMode, accessIdx) + epoch := rs.storeEpochs[storeIdx] + if atomic.CompareAndSwapUint32(&s.epoch, epoch, epoch+1) { + logutil.BgLogger().Info("mark store's regions need be refill", zap.String("store", s.addr)) + metrics.RegionCacheCounterWithInvalidateStoreRegionsOK.Inc() + } + // schedule a store addr resolve. + s.markNeedCheck(c.notifyCheckCh) + } + + // try next peer + rs.switchNextFlashPeer(r, accessIdx) + logutil.Logger(bo.GetCtx()).Info("switch region tiflash peer to next due to send request fail", + zap.Stringer("region", &ri.Region), + zap.Bool("needReload", scheduleReload), + zap.Error(err)) + + // force reload region when retry all known peers in region. + if scheduleReload { + r.scheduleReload() + } + } + } +} + // OnSendFail handles send request fail logic. func (c *RegionCache) OnSendFail(bo *Backoffer, ctx *RPCContext, scheduleReload bool, err error) { metrics.RegionCacheCounterWithSendFail.Inc() From 03589416f641cd00c6393f76468aa0339ec71e71 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 17 Jun 2021 19:02:38 +0800 Subject: [PATCH 07/27] expression: Support cast string as real push down (#25096) (#25109) --- expression/expr_to_pb_test.go | 5 +++++ expression/expression.go | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/expression/expr_to_pb_test.go b/expression/expr_to_pb_test.go index 35cd5dd9ab322..a649b5ad442a7 100644 --- a/expression/expr_to_pb_test.go +++ b/expression/expr_to_pb_test.go @@ -727,6 +727,11 @@ func (s *testEvaluatorSuite) TestExprPushDownToFlash(c *C) { c.Assert(err, IsNil) exprs = append(exprs, function) + // CastStringAsReal + function, err = NewFunction(mock.NewContext(), ast.Cast, types.NewFieldType(mysql.TypeDouble), stringColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + // Substring2ArgsUTF8 function, err = NewFunction(mock.NewContext(), ast.Substr, types.NewFieldType(mysql.TypeString), stringColumn, intColumn) c.Assert(err, IsNil) diff --git a/expression/expression.go b/expression/expression.go index 06b560f32d7b4..f5d301628dd76 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -1016,7 +1016,7 @@ func scalarExprSupportedByFlash(function *ScalarFunction) bool { switch function.Function.PbCode() { case tipb.ScalarFuncSig_CastIntAsInt, tipb.ScalarFuncSig_CastIntAsDecimal, tipb.ScalarFuncSig_CastIntAsString, tipb.ScalarFuncSig_CastIntAsTime, tipb.ScalarFuncSig_CastRealAsInt, tipb.ScalarFuncSig_CastRealAsDecimal, tipb.ScalarFuncSig_CastRealAsString, tipb.ScalarFuncSig_CastRealAsTime, - tipb.ScalarFuncSig_CastStringAsInt, tipb.ScalarFuncSig_CastStringAsDecimal, tipb.ScalarFuncSig_CastStringAsString, tipb.ScalarFuncSig_CastStringAsTime, + tipb.ScalarFuncSig_CastStringAsInt, tipb.ScalarFuncSig_CastStringAsDecimal, tipb.ScalarFuncSig_CastStringAsString, tipb.ScalarFuncSig_CastStringAsTime, tipb.ScalarFuncSig_CastStringAsReal, tipb.ScalarFuncSig_CastDecimalAsInt, tipb.ScalarFuncSig_CastDecimalAsDecimal, tipb.ScalarFuncSig_CastDecimalAsString, tipb.ScalarFuncSig_CastDecimalAsTime, tipb.ScalarFuncSig_CastTimeAsInt, tipb.ScalarFuncSig_CastTimeAsDecimal, tipb.ScalarFuncSig_CastTimeAsTime: return true From d2336242eafb9e5909a9352198670a300a2babba Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 17 Jun 2021 20:08:38 +0800 Subject: [PATCH 08/27] planner: support push down broadcast cartesian join to TiFlash (#25049) (#25106) --- go.mod | 2 +- go.sum | 4 +- planner/core/exhaust_physical_plans.go | 15 ++++- planner/core/plan_to_pb.go | 43 ++++++++---- .../testdata/integration_serial_suite_in.json | 6 +- .../integration_serial_suite_out.json | 65 ++++++++++++++++++- session/session.go | 1 + sessionctx/variable/session.go | 9 +++ sessionctx/variable/sysvar.go | 1 + sessionctx/variable/tidb_vars.go | 6 ++ 10 files changed, 132 insertions(+), 20 deletions(-) diff --git a/go.mod b/go.mod index 7627a7f68e232..1544d55f1f5de 100644 --- a/go.mod +++ b/go.mod @@ -53,7 +53,7 @@ require ( github.com/pingcap/parser v0.0.0-20210601031019-389d0cf6c5bd github.com/pingcap/sysutil v0.0.0-20210221112134-a07bda3bde99 github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible - github.com/pingcap/tipb v0.0.0-20210326161441-1164ca065d1b + github.com/pingcap/tipb v0.0.0-20210601083426-79a378b6d1c4 github.com/prometheus/client_golang v1.5.1 github.com/prometheus/client_model v0.2.0 github.com/prometheus/common v0.9.1 diff --git a/go.sum b/go.sum index 1cdae5347f28f..021bf6c261408 100644 --- a/go.sum +++ b/go.sum @@ -474,8 +474,8 @@ github.com/pingcap/sysutil v0.0.0-20210221112134-a07bda3bde99/go.mod h1:EB/852NM github.com/pingcap/tidb-dashboard v0.0.0-20210312062513-eef5d6404638/go.mod h1:OzFN8H0EDMMqeulPhPMw2i2JaiZWOKFQ7zdRPhENNgo= github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible h1:ceznmu/lLseGHP/jKyOa/3u/5H3wtLLLqkH2V3ssSjg= github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tipb v0.0.0-20210326161441-1164ca065d1b h1:sZHSH0mh8PcRbmZlsIqP7CEwnfFuBpmkGt5i9JStLWA= -github.com/pingcap/tipb v0.0.0-20210326161441-1164ca065d1b/go.mod h1:nsEhnMokcn7MRqd2J60yxpn/ac3ZH8A6GOJ9NslabUo= +github.com/pingcap/tipb v0.0.0-20210601083426-79a378b6d1c4 h1:n47+OwdI/uxKenfBT8Y2/be11MwbeLKNLdzOWnxNQKg= +github.com/pingcap/tipb v0.0.0-20210601083426-79a378b6d1c4/go.mod h1:nsEhnMokcn7MRqd2J60yxpn/ac3ZH8A6GOJ9NslabUo= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index fea4d3524c22d..78009f9f8afc3 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1636,6 +1636,9 @@ func (p *LogicalJoin) shouldUseMPPBCJ() bool { if p.ctx.GetSessionVars().BroadcastJoinThresholdSize == 0 || p.ctx.GetSessionVars().BroadcastJoinThresholdCount == 0 { return p.ctx.GetSessionVars().AllowBCJ } + if len(p.EqualConditions) == 0 && p.ctx.GetSessionVars().AllowCartesianBCJ == 2 { + return true + } if p.JoinType == LeftOuterJoin || p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin { return checkChildFitBC(p.children[1]) } else if p.JoinType == RightOuterJoin { @@ -1744,9 +1747,19 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC return nil } - if (p.JoinType != InnerJoin && p.JoinType != LeftOuterJoin && p.JoinType != RightOuterJoin && p.JoinType != SemiJoin && p.JoinType != AntiSemiJoin) || len(p.EqualConditions) == 0 { + if p.JoinType != InnerJoin && p.JoinType != LeftOuterJoin && p.JoinType != RightOuterJoin && p.JoinType != SemiJoin && p.JoinType != AntiSemiJoin { return nil } + + if len(p.EqualConditions) == 0 { + if p.ctx.GetSessionVars().AllowCartesianBCJ == 0 || !useBCJ { + return nil + } + } + if (len(p.LeftConditions) != 0 && p.JoinType != LeftOuterJoin) || (len(p.RightConditions) != 0 && p.JoinType != RightOuterJoin) { + return nil + } + if prop.PartitionTp == property.BroadcastType { return nil } diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index 185e750c0974c..bbaeac92ce5a7 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -366,7 +366,25 @@ func (p *PhysicalHashJoin) ToPB(ctx sessionctx.Context, storeType kv.StoreType) if err != nil { return nil, err } - otherConditions, err := expression.ExpressionsToPBList(sc, p.OtherConditions, client) + + var otherConditionsInJoin expression.CNFExprs + var otherEqConditionsFromIn expression.CNFExprs + if p.JoinType == AntiSemiJoin { + for _, condition := range p.OtherConditions { + if expression.IsEQCondFromIn(condition) { + otherEqConditionsFromIn = append(otherEqConditionsFromIn, condition) + } else { + otherConditionsInJoin = append(otherConditionsInJoin, condition) + } + } + } else { + otherConditionsInJoin = p.OtherConditions + } + otherConditions, err := expression.ExpressionsToPBList(sc, otherConditionsInJoin, client) + if err != nil { + return nil, err + } + otherEqConditions, err := expression.ExpressionsToPBList(sc, otherEqConditionsFromIn, client) if err != nil { return nil, err } @@ -397,17 +415,18 @@ func (p *PhysicalHashJoin) ToPB(ctx sessionctx.Context, storeType kv.StoreType) buildFiledTypes = append(buildFiledTypes, expression.ToPBFieldType(retType)) } join := &tipb.Join{ - JoinType: pbJoinType, - JoinExecType: tipb.JoinExecType_TypeHashJoin, - InnerIdx: int64(p.InnerChildIdx), - LeftJoinKeys: left, - RightJoinKeys: right, - ProbeTypes: probeFiledTypes, - BuildTypes: buildFiledTypes, - LeftConditions: leftConditions, - RightConditions: rightConditions, - OtherConditions: otherConditions, - Children: []*tipb.Executor{lChildren, rChildren}, + JoinType: pbJoinType, + JoinExecType: tipb.JoinExecType_TypeHashJoin, + InnerIdx: int64(p.InnerChildIdx), + LeftJoinKeys: left, + RightJoinKeys: right, + ProbeTypes: probeFiledTypes, + BuildTypes: buildFiledTypes, + LeftConditions: leftConditions, + RightConditions: rightConditions, + OtherConditions: otherConditions, + OtherEqConditionsFromIn: otherEqConditions, + Children: []*tipb.Executor{lChildren, rChildren}, } executorID := p.ExplainID().String() diff --git a/planner/core/testdata/integration_serial_suite_in.json b/planner/core/testdata/integration_serial_suite_in.json index f9a7a9130f742..de096cad9e24a 100644 --- a/planner/core/testdata/integration_serial_suite_in.json +++ b/planner/core/testdata/integration_serial_suite_in.json @@ -31,7 +31,11 @@ "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k)", "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k)", - "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)" + "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", + "explain format = 'brief' select count(*) from fact_t join d1_t on fact_t.d1_k > d1_t.d1_k", + "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k > d1_t.d1_k", + "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k > d1_t.d1_k", + "explain format = 'brief' select count(*) from fact_t where d1_k not in (select d1_k from d1_t)" ] }, { diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index 673315a7d9f2d..e792f2d581f0b 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -285,6 +285,65 @@ " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", " └─TableFullScan(Probe) 8.00 cop[tiflash] table:fact_t keep order:false" ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t join d1_t on fact_t.d1_k > d1_t.d1_k", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 16.00 batchCop[tiflash] CARTESIAN inner join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", + " ├─ExchangeReceiver(Build) 2.00 batchCop[tiflash] ", + " │ └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k > d1_t.d1_k", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 16.00 batchCop[tiflash] CARTESIAN left outer join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", + " ├─ExchangeReceiver(Build) 2.00 batchCop[tiflash] ", + " │ └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false", + " └─TableFullScan(Probe) 8.00 batchCop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k > d1_t.d1_k", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 16.00 batchCop[tiflash] CARTESIAN right outer join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", + " ├─ExchangeReceiver(Build) 8.00 batchCop[tiflash] ", + " │ └─ExchangeSender 8.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " │ └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false", + " └─TableFullScan(Probe) 2.00 batchCop[tiflash] table:d1_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t where d1_k not in (select d1_k from d1_t)", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 6.40 root data:ExchangeSender", + " └─ExchangeSender 6.40 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 6.40 cop[tiflash] CARTESIAN anti semi join, other cond:eq(test.fact_t.d1_k, test.d1_t.d1_k)", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", + " └─TableFullScan(Probe) 8.00 cop[tiflash] table:fact_t keep order:false" + ] } ] }, @@ -2318,11 +2377,11 @@ "HashAgg 7992.00 root group by:test.ts.col_char_64, funcs:firstrow(test.ts.col_char_64)->test.ts.col_char_64", "└─HashJoin 9990.00 root CARTESIAN inner join, other cond:or(ge(test.ts.col_char_64_not_null, Column#25), if(ne(Column#26, 0), NULL, 0))", " ├─Selection(Build) 0.80 root ne(Column#27, 0)", - " │ └─HashAgg 1.00 root funcs:min(Column#33)->Column#25, funcs:sum(Column#34)->Column#26, funcs:count(Column#35)->Column#27", + " │ └─HashAgg 1.00 root funcs:min(Column#36)->Column#25, funcs:sum(Column#37)->Column#26, funcs:count(Column#38)->Column#27", " │ └─TableReader 1.00 root data:ExchangeSender", " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " │ └─HashAgg 1.00 batchCop[tiflash] funcs:min(Column#39)->Column#33, funcs:sum(Column#40)->Column#34, funcs:count(1)->Column#35", - " │ └─Projection 10000.00 batchCop[tiflash] test.ts.col_varchar_64, cast(isnull(test.ts.col_varchar_64), decimal(22,0) BINARY)->Column#40", + " │ └─HashAgg 1.00 batchCop[tiflash] funcs:min(Column#42)->Column#36, funcs:sum(Column#43)->Column#37, funcs:count(1)->Column#38", + " │ └─Projection 10000.00 batchCop[tiflash] test.ts.col_varchar_64, cast(isnull(test.ts.col_varchar_64), decimal(22,0) BINARY)->Column#43", " │ └─TableFullScan 10000.00 batchCop[tiflash] table:SUBQUERY4_t1 keep order:false, stats:pseudo", " └─TableReader(Probe) 12487.50 root data:ExchangeSender", " └─ExchangeSender 12487.50 cop[tiflash] ExchangeType: PassThrough", diff --git a/session/session.go b/session/session.go index d1666be45063f..76fac6d932b61 100644 --- a/session/session.go +++ b/session/session.go @@ -2506,6 +2506,7 @@ var builtinGlobalVariable = []string{ variable.TiDBOptBCJ, variable.TiDBBCJThresholdSize, variable.TiDBBCJThresholdCount, + variable.TiDBOptCartesianBCJ, variable.TiDBRowFormatVersion, variable.TiDBEnableStmtSummary, variable.TiDBStmtSummaryInternalQuery, diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 82546fd553d65..137a50891981d 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -482,6 +482,12 @@ type SessionVars struct { // AllowBCJ means allow broadcast join. AllowBCJ bool + + // AllowCartesianBCJ means allow broadcast CARTESIAN join, 0 means not allow, 1 means allow broadcast CARTESIAN join + // but the table size should under the broadcast threshold, 2 means allow broadcast CARTESIAN join even if the table + // size exceeds the broadcast threshold + AllowCartesianBCJ int + // AllowDistinctAggPushDown can be set true to allow agg with distinct push down to tikv/tiflash. AllowDistinctAggPushDown bool @@ -953,6 +959,7 @@ func NewSessionVars() *SessionVars { StmtCtx: new(stmtctx.StatementContext), AllowAggPushDown: false, AllowBCJ: false, + AllowCartesianBCJ: DefOptCartesianBCJ, BroadcastJoinThresholdSize: DefBroadcastJoinThresholdSize, BroadcastJoinThresholdCount: DefBroadcastJoinThresholdSize, OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, @@ -1442,6 +1449,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { s.BroadcastJoinThresholdSize = tidbOptInt64(val, DefBroadcastJoinThresholdSize) case TiDBBCJThresholdCount: s.BroadcastJoinThresholdCount = tidbOptInt64(val, DefBroadcastJoinThresholdCount) + case TiDBOptCartesianBCJ: + s.AllowCartesianBCJ = int(tidbOptInt64(val, DefOptCartesianBCJ)) case TiDBOptDistinctAggPushDown: s.AllowDistinctAggPushDown = TiDBOptOn(val) case TiDBOptWriteRowID: diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 29a073936645c..88001e68e53e0 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -594,6 +594,7 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeSession, Name: TiDBOptDistinctAggPushDown, Value: BoolToOnOff(config.GetGlobalConfig().Performance.DistinctAggPushDown), Type: TypeBool}, {Scope: ScopeSession, Name: TiDBOptWriteRowID, Value: BoolToOnOff(DefOptWriteRowID)}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBBuildStatsConcurrency, Value: strconv.Itoa(DefBuildStatsConcurrency)}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCartesianBCJ, Value: strconv.Itoa(DefOptCartesianBCJ), Type: TypeInt, MinValue: 0, MaxValue: 2}, {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeRatio, Value: strconv.FormatFloat(DefAutoAnalyzeRatio, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64}, {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeStartTime, Value: DefAutoAnalyzeStartTime, Type: TypeTime}, {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeEndTime, Value: DefAutoAnalyzeEndTime, Type: TypeTime}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 144d09a52d929..02727ac14ec60 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -47,7 +47,12 @@ const ( // tidb_opt_agg_push_down is used to enable/disable the optimizer rule of aggregation push down. TiDBOptAggPushDown = "tidb_opt_agg_push_down" + // TiDBOptBCJ is used to enable/disable broadcast join in MPP mode TiDBOptBCJ = "tidb_opt_broadcast_join" + + // TiDBOptCartesianBCJ is used to disable/enable broadcast cartesian join in MPP mode + TiDBOptCartesianBCJ = "tidb_opt_broadcast_cartesian_join" + // tidb_opt_distinct_agg_push_down is used to decide whether agg with distinct should be pushed to tikv/tiflash. TiDBOptDistinctAggPushDown = "tidb_opt_distinct_agg_push_down" @@ -566,6 +571,7 @@ const ( DefSkipASCIICheck = false DefOptAggPushDown = false DefOptBCJ = false + DefOptCartesianBCJ = 1 DefOptWriteRowID = false DefOptCorrelationThreshold = 0.9 DefOptCorrelationExpFactor = 1 From 2c4be621b7a7638f2f24fd89df7a6ac8592c1fe0 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 18 Jun 2021 11:14:38 +0800 Subject: [PATCH 09/27] executor: fix data race of parallel apply operator (#24257) (#24345) --- executor/parallel_apply.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/executor/parallel_apply.go b/executor/parallel_apply.go index 8c215e0bddb60..39c7ea0da3a9a 100644 --- a/executor/parallel_apply.go +++ b/executor/parallel_apply.go @@ -159,12 +159,14 @@ func (e *ParallelNestedLoopApplyExec) Next(ctx context.Context, req *chunk.Chunk // Close implements the Executor interface. func (e *ParallelNestedLoopApplyExec) Close() error { e.memTracker = nil - err := e.outerExec.Close() if atomic.LoadUint32(&e.started) == 1 { close(e.exit) e.notifyWg.Wait() e.started = 0 } + // Wait all workers to finish before Close() is called. + // Otherwise we may got data race. + err := e.outerExec.Close() if e.runtimeStats != nil { runtimeStats := newJoinRuntimeStats() From 4c5046f5634e2a06791b48d83647538847366d44 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 18 Jun 2021 11:52:38 +0800 Subject: [PATCH 10/27] expression: support `str_to_date` push to TiFlash (#25095) (#25148) --- expression/expr_to_pb_test.go | 6 ++++++ expression/expression.go | 9 +++++++++ 2 files changed, 15 insertions(+) diff --git a/expression/expr_to_pb_test.go b/expression/expr_to_pb_test.go index a649b5ad442a7..c5d95c5434b58 100644 --- a/expression/expr_to_pb_test.go +++ b/expression/expr_to_pb_test.go @@ -742,6 +742,12 @@ func (s *testEvaluatorSuite) TestExprPushDownToFlash(c *C) { c.Assert(err, IsNil) exprs = append(exprs, function) + // StrToDateDateTime + function, err = NewFunction(mock.NewContext(), ast.StrToDate, types.NewFieldType(mysql.TypeDatetime), stringColumn, stringColumn) + c.Assert(err, IsNil) + c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_StrToDateDatetime) + exprs = append(exprs, function) + function, err = NewFunction(mock.NewContext(), ast.Round, types.NewFieldType(mysql.TypeDouble), realColumn) c.Assert(err, IsNil) exprs = append(exprs, function) diff --git a/expression/expression.go b/expression/expression.go index f5d301628dd76..5cc6c82e8e853 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -1036,6 +1036,15 @@ func scalarExprSupportedByFlash(function *ScalarFunction) bool { case tipb.ScalarFuncSig_ExtractDatetime: return true } + case ast.StrToDate: + switch function.Function.PbCode() { + case + tipb.ScalarFuncSig_StrToDateDate, + tipb.ScalarFuncSig_StrToDateDatetime: + return true + default: + return false + } } return false } From fbe99ffa9100f08e139e0fd27ab8e9ad3f9e8ff8 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 18 Jun 2021 14:42:38 +0800 Subject: [PATCH 11/27] planner: Mpp outer join build side (#25130) (#25142) --- executor/tiflash_test.go | 27 +++ planner/core/exhaust_physical_plans.go | 16 +- planner/core/integration_test.go | 132 +++++++++++++ .../testdata/integration_serial_suite_in.json | 21 +++ .../integration_serial_suite_out.json | 177 ++++++++++++++---- session/session.go | 1 + sessionctx/variable/session.go | 6 + sessionctx/variable/sysvar.go | 1 + sessionctx/variable/tidb_vars.go | 4 + 9 files changed, 349 insertions(+), 36 deletions(-) diff --git a/executor/tiflash_test.go b/executor/tiflash_test.go index c09d70cde0926..0c2763d299148 100644 --- a/executor/tiflash_test.go +++ b/executor/tiflash_test.go @@ -85,6 +85,9 @@ func (s *tiflashTestSuite) TestReadPartitionTable(c *C) { tk.MustExec("insert into t values(2,0)") tk.MustExec("insert into t values(3,0)") tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") tk.MustQuery("select /*+ STREAM_AGG() */ count(*) from t").Check(testkit.Rows("3")) tk.MustQuery("select * from t order by a").Check(testkit.Rows("1 0", "2 0", "3 0")) @@ -129,6 +132,9 @@ func (s *tiflashTestSuite) TestReadUnsigedPK(c *C) { tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") tk.MustExec("set @@session.tidb_allow_mpp=ON") tk.MustExec("set @@session.tidb_opt_broadcast_join=ON") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") tk.MustQuery("select count(*) from t1 , t where t1.a = t.a").Check(testkit.Rows("5")) tk.MustQuery("select count(*) from t1 , t where t1.a = t.a and ((t1.a < 9223372036854775800 and t1.a > 2) or (t1.a <= 1 and t1.a > -1))").Check(testkit.Rows("3")) @@ -161,6 +167,9 @@ func (s *tiflashTestSuite) TestMppExecution(c *C) { tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") tk.MustExec("set @@session.tidb_allow_mpp=ON") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") for i := 0; i < 20; i++ { // test if it is stable. tk.MustQuery("select count(*) from t1 , t where t1.a = t.a").Check(testkit.Rows("3")) @@ -267,6 +276,9 @@ func (s *tiflashTestSuite) TestPartitionTable(c *C) { failpoint.Enable("github.com/pingcap/tidb/executor/checkUseMPP", `return(true)`) tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") tk.MustExec("set @@session.tidb_allow_mpp=ON") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") failpoint.Enable("github.com/pingcap/tidb/executor/checkTotalMPPTasks", `return(4)`) tk.MustQuery("select count(*) from t").Check(testkit.Rows("4")) failpoint.Disable("github.com/pingcap/tidb/executor/checkTotalMPPTasks") @@ -351,6 +363,9 @@ func (s *tiflashTestSuite) TestMppEnum(c *C) { tk.MustExec("insert into t values(3,'zca')") tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") tk.MustExec("set @@session.tidb_allow_mpp=ON") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") tk.MustQuery("select t1.b from t t1 join t t2 on t1.a = t2.a order by t1.b").Check(testkit.Rows("aca", "bca", "zca")) } @@ -372,6 +387,9 @@ func (s *tiflashTestSuite) TestCancelMppTasks(c *C) { c.Assert(err, IsNil) tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") tk.MustExec("set @@session.tidb_allow_mpp=ON") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") atomic.StoreUint32(&tk.Se.GetSessionVars().Killed, 0) c.Assert(failpoint.Enable(hang, `return(true)`), IsNil) wg := &sync.WaitGroup{} @@ -416,6 +434,9 @@ func (s *tiflashTestSuite) TestMppGoroutinesExitFromErrors(c *C) { tk.MustExec("insert into t1 values(3,0)") tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") tk.MustExec("set @@session.tidb_allow_mpp=ON") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") c.Assert(failpoint.Enable(mppNonRootTaskError, `return(true)`), IsNil) c.Assert(failpoint.Enable(hang, `return(true)`), IsNil) @@ -447,6 +468,9 @@ func (s *tiflashTestSuite) TestMppApply(c *C) { tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") tk.MustExec("set @@session.tidb_allow_mpp=ON") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") // table full scan with correlated filter tk.MustQuery("select /*+ agg_to_cop(), hash_agg()*/ count(*) from x1 where a >= any (select a from x2 where x1.a = x2.a) order by 1;").Check(testkit.Rows("3")) // table range scan with correlated access conditions @@ -480,6 +504,9 @@ func (s *tiflashTestSuite) TestTiFlashVirtualColumn(c *C) { tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") tk.MustExec("set @@session.tidb_allow_mpp=ON") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") tk.MustQuery("select /*+ hash_agg() */ count(*) from t1 where c > b'01'").Check(testkit.Rows("2")) tk.MustQuery("select /*+ hash_agg() */ count(*) from t2 where c > 1").Check(testkit.Rows("2")) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 78009f9f8afc3..b3f10d6bfabc7 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1783,9 +1783,23 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC if p.children[0].statsInfo().Count() > p.children[1].statsInfo().Count() { preferredBuildIndex = 1 } - } else if p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin || p.JoinType == LeftOuterJoin { + } else if p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin { preferredBuildIndex = 1 } + if p.JoinType == LeftOuterJoin || p.JoinType == RightOuterJoin { + // TiFlash does not requires that the build side must be the inner table for outer join + // so we can choose the build side based on the row count, except that + // 1. it is a broadcast join(for broadcast join, it make sense to use the broadcast side as the build side) + // 2. or session variable MPPOuterJoinFixedBuildSide is set to true + // 3. or there are otherConditions for this join + if useBCJ || p.ctx.GetSessionVars().MPPOuterJoinFixedBuildSide || len(p.OtherConditions) > 0 { + if p.JoinType == LeftOuterJoin { + preferredBuildIndex = 1 + } + } else if p.children[0].statsInfo().Count() > p.children[1].statsInfo().Count() { + preferredBuildIndex = 1 + } + } baseJoin.InnerChildIdx = preferredBuildIndex childrenProps := make([]*property.PhysicalProperty, 2) if useBCJ { diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 39613616dcaa5..43e08b735a6a8 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -462,6 +462,138 @@ func (s *testIntegrationSerialSuite) TestMPPJoin(c *C) { } } +func (s *testIntegrationSerialSuite) TestMPPOuterJoinBuildSideForBroadcastJoin(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists a") + tk.MustExec("create table a(id int, value int)") + tk.MustExec("insert into a values(1,2),(2,3)") + tk.MustExec("analyze table a") + tk.MustExec("drop table if exists b") + tk.MustExec("create table b(id int, value int)") + tk.MustExec("insert into b values(1,2),(2,3),(3,4)") + tk.MustExec("analyze table b") + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Se) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + c.Assert(exists, IsTrue) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "a" || tblInfo.Name.L == "b" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.tidb_opt_mpp_outer_join_fixed_build_side = 0") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 10000") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 10000") + var input []string + var output []struct { + SQL string + Plan []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + } +} + +func (s *testIntegrationSerialSuite) TestMPPOuterJoinBuildSideForShuffleJoinWithFixedBuildSide(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists a") + tk.MustExec("create table a(id int, value int)") + tk.MustExec("insert into a values(1,2),(2,3)") + tk.MustExec("analyze table a") + tk.MustExec("drop table if exists b") + tk.MustExec("create table b(id int, value int)") + tk.MustExec("insert into b values(1,2),(2,3),(3,4)") + tk.MustExec("analyze table b") + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Se) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + c.Assert(exists, IsTrue) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "a" || tblInfo.Name.L == "b" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.tidb_opt_mpp_outer_join_fixed_build_side = 1") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 0") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 0") + var input []string + var output []struct { + SQL string + Plan []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + } +} + +func (s *testIntegrationSerialSuite) TestMPPOuterJoinBuildSideForShuffleJoin(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists a") + tk.MustExec("create table a(id int, value int)") + tk.MustExec("insert into a values(1,2),(2,3)") + tk.MustExec("analyze table a") + tk.MustExec("drop table if exists b") + tk.MustExec("create table b(id int, value int)") + tk.MustExec("insert into b values(1,2),(2,3),(3,4)") + tk.MustExec("analyze table b") + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Se) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + c.Assert(exists, IsTrue) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "a" || tblInfo.Name.L == "b" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.tidb_opt_mpp_outer_join_fixed_build_side = 0") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 0") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 0") + var input []string + var output []struct { + SQL string + Plan []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + } +} + func (s *testIntegrationSerialSuite) TestMPPShuffledJoin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/planner/core/testdata/integration_serial_suite_in.json b/planner/core/testdata/integration_serial_suite_in.json index de096cad9e24a..30bf481a15aca 100644 --- a/planner/core/testdata/integration_serial_suite_in.json +++ b/planner/core/testdata/integration_serial_suite_in.json @@ -38,6 +38,27 @@ "explain format = 'brief' select count(*) from fact_t where d1_k not in (select d1_k from d1_t)" ] }, + { + "name": "TestMPPOuterJoinBuildSideForBroadcastJoin", + "cases": [ + "explain format = 'brief' select count(*) from a left join b on a.id = b.id", + "explain format = 'brief' select count(*) from b right join a on a.id = b.id" + ] + }, + { + "name": "TestMPPOuterJoinBuildSideForShuffleJoinWithFixedBuildSide", + "cases": [ + "explain format = 'brief' select count(*) from a left join b on a.id = b.id", + "explain format = 'brief' select count(*) from b right join a on a.id = b.id" + ] + }, + { + "name": "TestMPPOuterJoinBuildSideForShuffleJoin", + "cases": [ + "explain format = 'brief' select count(*) from a left join b on a.id = b.id", + "explain format = 'brief' select count(*) from b right join a on a.id = b.id" + ] + }, { "name": "TestMPPShuffledJoin", "cases": [ diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index e792f2d581f0b..2d03c51d31e7c 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -347,6 +347,113 @@ } ] }, + { + "Name": "TestMPPOuterJoinBuildSideForBroadcastJoin", + "Cases": [ + { + "SQL": "explain format = 'brief' select count(*) from a left join b on a.id = b.id", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] left outer join, equal:[eq(test.a.id, test.b.id)]", + " ├─ExchangeReceiver(Build) 3.00 cop[tiflash] ", + " │ └─ExchangeSender 3.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", + " │ └─TableFullScan 3.00 cop[tiflash] table:b keep order:false", + " └─TableFullScan(Probe) 2.00 cop[tiflash] table:a keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from b right join a on a.id = b.id", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] right outer join, equal:[eq(test.b.id, test.a.id)]", + " ├─ExchangeReceiver(Build) 3.00 cop[tiflash] ", + " │ └─ExchangeSender 3.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", + " │ └─TableFullScan 3.00 cop[tiflash] table:b keep order:false", + " └─TableFullScan(Probe) 2.00 cop[tiflash] table:a keep order:false" + ] + } + ] + }, + { + "Name": "TestMPPOuterJoinBuildSideForShuffleJoinWithFixedBuildSide", + "Cases": [ + { + "SQL": "explain format = 'brief' select count(*) from a left join b on a.id = b.id", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] left outer join, equal:[eq(test.a.id, test.b.id)]", + " ├─ExchangeReceiver(Build) 3.00 cop[tiflash] ", + " │ └─ExchangeSender 3.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.b.id", + " │ └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", + " │ └─TableFullScan 3.00 cop[tiflash] table:b keep order:false", + " └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.a.id", + " └─TableFullScan 2.00 cop[tiflash] table:a keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from b right join a on a.id = b.id", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] right outer join, equal:[eq(test.b.id, test.a.id)]", + " ├─ExchangeReceiver(Build) 3.00 cop[tiflash] ", + " │ └─ExchangeSender 3.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.b.id", + " │ └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", + " │ └─TableFullScan 3.00 cop[tiflash] table:b keep order:false", + " └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.a.id", + " └─TableFullScan 2.00 cop[tiflash] table:a keep order:false" + ] + } + ] + }, + { + "Name": "TestMPPOuterJoinBuildSideForShuffleJoin", + "Cases": [ + { + "SQL": "explain format = 'brief' select count(*) from a left join b on a.id = b.id", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] left outer join, equal:[eq(test.a.id, test.b.id)]", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.a.id", + " │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", + " └─ExchangeReceiver(Probe) 3.00 cop[tiflash] ", + " └─ExchangeSender 3.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.b.id", + " └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", + " └─TableFullScan 3.00 cop[tiflash] table:b keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from b right join a on a.id = b.id", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] right outer join, equal:[eq(test.b.id, test.a.id)]", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.a.id", + " │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", + " └─ExchangeReceiver(Probe) 3.00 cop[tiflash] ", + " └─ExchangeSender 3.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.b.id", + " └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", + " └─TableFullScan 3.00 cop[tiflash] table:b keep order:false" + ] + } + ] + }, { "Name": "TestMPPShuffledJoin", "Cases": [ @@ -471,13 +578,13 @@ " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", " └─HashJoin 32.00 batchCop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 16.00 batchCop[tiflash] ", - " │ └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d1_k", - " │ └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false", - " └─ExchangeReceiver(Probe) 4.00 batchCop[tiflash] ", - " └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d1_t.d1_k", - " └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false" + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d1_t.d1_k", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d1_k", + " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" ] }, { @@ -565,13 +672,13 @@ " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", " └─HashJoin 32.00 batchCop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], right cond:gt(test.d1_t.value, 10)", - " ├─ExchangeReceiver(Build) 16.00 batchCop[tiflash] ", - " │ └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d1_k", - " │ └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false", - " └─ExchangeReceiver(Probe) 4.00 batchCop[tiflash] ", - " └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d1_t.d1_k", - " └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false" + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d1_t.d1_k", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d1_k", + " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" ] }, { @@ -1810,27 +1917,27 @@ "└─TableReader 19492.21 root data:ExchangeSender", " └─ExchangeSender 19492.21 cop[tiflash] ExchangeType: PassThrough", " └─HashJoin 19492.21 cop[tiflash] right outer join, equal:[eq(test.t.c3, test.t.c4)]", - " ├─Projection(Build) 15593.77 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", - " │ └─HashJoin 15593.77 cop[tiflash] inner join, equal:[eq(test.t.c5, test.t.c3)]", - " │ ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", - " │ │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#25", - " │ │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c3, decimal(40,20))->Column#25", - " │ │ └─TableFullScan 10000.00 cop[tiflash] table:t3 keep order:false, stats:pseudo", - " │ └─ExchangeReceiver(Probe) 12475.01 cop[tiflash] ", - " │ └─ExchangeSender 12475.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c5", - " │ └─HashJoin 12475.01 cop[tiflash] inner join, equal:[eq(test.t.c2, test.t.c1)]", - " │ ├─ExchangeReceiver(Build) 9980.01 cop[tiflash] ", - " │ │ └─ExchangeSender 9980.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c2", - " │ │ └─Selection 9980.01 cop[tiflash] not(isnull(test.t.c2)), not(isnull(test.t.c5))", - " │ │ └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo", - " │ └─ExchangeReceiver(Probe) 9990.00 cop[tiflash] ", - " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c1", - " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.c1))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 10000.00 cop[tiflash] ", - " └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#27", - " └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c4, decimal(40,20))->Column#27", - " └─TableFullScan 10000.00 cop[tiflash] table:t4 keep order:false, stats:pseudo" + " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", + " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#27", + " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c4, decimal(40,20))->Column#27", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t4 keep order:false, stats:pseudo", + " └─Projection(Probe) 15593.77 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", + " └─HashJoin 15593.77 cop[tiflash] inner join, equal:[eq(test.t.c5, test.t.c3)]", + " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", + " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#25", + " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c3, decimal(40,20))->Column#25", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t3 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 12475.01 cop[tiflash] ", + " └─ExchangeSender 12475.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c5", + " └─HashJoin 12475.01 cop[tiflash] inner join, equal:[eq(test.t.c2, test.t.c1)]", + " ├─ExchangeReceiver(Build) 9980.01 cop[tiflash] ", + " │ └─ExchangeSender 9980.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c2", + " │ └─Selection 9980.01 cop[tiflash] not(isnull(test.t.c2)), not(isnull(test.t.c5))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 cop[tiflash] ", + " └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c1", + " └─Selection 9990.00 cop[tiflash] not(isnull(test.t.c1))", + " └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { diff --git a/session/session.go b/session/session.go index 76fac6d932b61..19ac9fcf8e8b2 100644 --- a/session/session.go +++ b/session/session.go @@ -2507,6 +2507,7 @@ var builtinGlobalVariable = []string{ variable.TiDBBCJThresholdSize, variable.TiDBBCJThresholdCount, variable.TiDBOptCartesianBCJ, + variable.TiDBOptMPPOuterJoinFixedBuildSide, variable.TiDBRowFormatVersion, variable.TiDBEnableStmtSummary, variable.TiDBStmtSummaryInternalQuery, diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 137a50891981d..1c333a9bab074 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -488,6 +488,9 @@ type SessionVars struct { // size exceeds the broadcast threshold AllowCartesianBCJ int + // MPPOuterJoinFixedBuildSide means in MPP plan, always use inner table as build side for out join + MPPOuterJoinFixedBuildSide bool + // AllowDistinctAggPushDown can be set true to allow agg with distinct push down to tikv/tiflash. AllowDistinctAggPushDown bool @@ -960,6 +963,7 @@ func NewSessionVars() *SessionVars { AllowAggPushDown: false, AllowBCJ: false, AllowCartesianBCJ: DefOptCartesianBCJ, + MPPOuterJoinFixedBuildSide: DefOptMPPOuterJoinFixedBuildSide, BroadcastJoinThresholdSize: DefBroadcastJoinThresholdSize, BroadcastJoinThresholdCount: DefBroadcastJoinThresholdSize, OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, @@ -1451,6 +1455,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { s.BroadcastJoinThresholdCount = tidbOptInt64(val, DefBroadcastJoinThresholdCount) case TiDBOptCartesianBCJ: s.AllowCartesianBCJ = int(tidbOptInt64(val, DefOptCartesianBCJ)) + case TiDBOptMPPOuterJoinFixedBuildSide: + s.MPPOuterJoinFixedBuildSide = TiDBOptOn(val) case TiDBOptDistinctAggPushDown: s.AllowDistinctAggPushDown = TiDBOptOn(val) case TiDBOptWriteRowID: diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 88001e68e53e0..95c444a911137 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -595,6 +595,7 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeSession, Name: TiDBOptWriteRowID, Value: BoolToOnOff(DefOptWriteRowID)}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBBuildStatsConcurrency, Value: strconv.Itoa(DefBuildStatsConcurrency)}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCartesianBCJ, Value: strconv.Itoa(DefOptCartesianBCJ), Type: TypeInt, MinValue: 0, MaxValue: 2}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptMPPOuterJoinFixedBuildSide, Value: BoolToOnOff(DefOptMPPOuterJoinFixedBuildSide), Type: TypeBool}, {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeRatio, Value: strconv.FormatFloat(DefAutoAnalyzeRatio, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64}, {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeStartTime, Value: DefAutoAnalyzeStartTime, Type: TypeTime}, {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeEndTime, Value: DefAutoAnalyzeEndTime, Type: TypeTime}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 02727ac14ec60..8acc2bb02586e 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -53,6 +53,9 @@ const ( // TiDBOptCartesianBCJ is used to disable/enable broadcast cartesian join in MPP mode TiDBOptCartesianBCJ = "tidb_opt_broadcast_cartesian_join" + // TiDBOptMPPOuterJoinFixedBuildSide is set to true, then in MPP plan, always use inner table as build side for out join + TiDBOptMPPOuterJoinFixedBuildSide = "tidb_opt_mpp_outer_join_fixed_build_side" + // tidb_opt_distinct_agg_push_down is used to decide whether agg with distinct should be pushed to tikv/tiflash. TiDBOptDistinctAggPushDown = "tidb_opt_distinct_agg_push_down" @@ -572,6 +575,7 @@ const ( DefOptAggPushDown = false DefOptBCJ = false DefOptCartesianBCJ = 1 + DefOptMPPOuterJoinFixedBuildSide = false DefOptWriteRowID = false DefOptCorrelationThreshold = 0.9 DefOptCorrelationExpFactor = 1 From eb9b926116694fcabaa9010db621e3d492fbb3e3 Mon Sep 17 00:00:00 2001 From: Zhi Qi <30543181+LittleFall@users.noreply.github.com> Date: Fri, 18 Jun 2021 16:36:38 +0800 Subject: [PATCH 12/27] expression: [cherry-pick-5.0] support push fucntions down to TiFlash: unix_timestamp,concat,year,day,datediff,datesub,castTimeAsString,concat_ws. (#25564) --- expression/expr_to_pb_test.go | 73 ++++++++++++++++++++++++++++++++++- expression/expression.go | 19 +++++++-- 2 files changed, 87 insertions(+), 5 deletions(-) diff --git a/expression/expr_to_pb_test.go b/expression/expr_to_pb_test.go index c5d95c5434b58..fa3713af23537 100644 --- a/expression/expr_to_pb_test.go +++ b/expression/expr_to_pb_test.go @@ -748,12 +748,83 @@ func (s *testEvaluatorSuite) TestExprPushDownToFlash(c *C) { c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_StrToDateDatetime) exprs = append(exprs, function) + // ScalarFuncSig_RoundReal function, err = NewFunction(mock.NewContext(), ast.Round, types.NewFieldType(mysql.TypeDouble), realColumn) c.Assert(err, IsNil) exprs = append(exprs, function) - function, err = NewFunction(mock.NewContext(), ast.Round, types.NewFieldType(mysql.TypeLonglong), intColumn) + // Year + function, err = NewFunction(mock.NewContext(), ast.Year, types.NewFieldType(mysql.TypeLonglong), datetimeColumn) c.Assert(err, IsNil) + c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_Year) + exprs = append(exprs, function) + + // Day + function, err = NewFunction(mock.NewContext(), ast.Day, types.NewFieldType(mysql.TypeLonglong), datetimeColumn) + c.Assert(err, IsNil) + c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_DayOfMonth) + exprs = append(exprs, function) + + // Datediff + function, err = NewFunction(mock.NewContext(), ast.DateDiff, types.NewFieldType(mysql.TypeLonglong), datetimeColumn, datetimeColumn) + c.Assert(err, IsNil) + c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_DateDiff) + exprs = append(exprs, function) + + // Datesub + function, err = NewFunction(mock.NewContext(), ast.DateSub, types.NewFieldType(mysql.TypeDatetime), datetimeColumn, intColumn, stringColumn) + c.Assert(err, IsNil) + c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_SubDateDatetimeInt) + exprs = append(exprs, function) + function, err = NewFunction(mock.NewContext(), ast.DateSub, types.NewFieldType(mysql.TypeDatetime), stringColumn, intColumn, stringColumn) + c.Assert(err, IsNil) + c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_SubDateStringInt) + exprs = append(exprs, function) + function, err = NewFunction(mock.NewContext(), ast.SubDate, types.NewFieldType(mysql.TypeDatetime), datetimeColumn, intColumn, stringColumn) + c.Assert(err, IsNil) + c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_SubDateDatetimeInt) + exprs = append(exprs, function) + + // castTimeAsString: + function, err = NewFunction(mock.NewContext(), ast.Cast, types.NewFieldType(mysql.TypeString), datetimeColumn) + c.Assert(err, IsNil) + c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_CastTimeAsString) + exprs = append(exprs, function) + + // concat_ws + function, err = NewFunction(mock.NewContext(), ast.ConcatWS, types.NewFieldType(mysql.TypeString), stringColumn, stringColumn, stringColumn) + c.Assert(err, IsNil) + c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_ConcatWS) + exprs = append(exprs, function) + + // StrToDateDateTime + function, err = NewFunction(mock.NewContext(), ast.StrToDate, types.NewFieldType(mysql.TypeDatetime), stringColumn, stringColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + + // concat + function, err = NewFunction(mock.NewContext(), ast.Concat, types.NewFieldType(mysql.TypeString), stringColumn, intColumn, realColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + + // UnixTimestampCurrent + function, err = NewFunction(mock.NewContext(), ast.UnixTimestamp, types.NewFieldType(mysql.TypeLonglong)) + c.Assert(err, IsNil) + _, ok := function.(*Constant) + c.Assert(ok, IsTrue) + + // UnixTimestampInt + datetimeColumn.RetType.Decimal = 0 + function, err = NewFunction(mock.NewContext(), ast.UnixTimestamp, types.NewFieldType(mysql.TypeLonglong), datetimeColumn) + c.Assert(err, IsNil) + c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_UnixTimestampInt) + exprs = append(exprs, function) + + // UnixTimestampDecimal + datetimeColumn.RetType.Decimal = types.UnspecifiedLength + function, err = NewFunction(mock.NewContext(), ast.UnixTimestamp, types.NewFieldType(mysql.TypeNewDecimal), datetimeColumn) + c.Assert(err, IsNil) + c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_UnixTimestampDec) exprs = append(exprs, function) canPush := CanExprsPushDown(sc, exprs, client, kv.TiFlash) diff --git a/expression/expression.go b/expression/expression.go index 5cc6c82e8e853..862a53a1290d1 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -1001,8 +1001,9 @@ func scalarExprSupportedByFlash(function *ScalarFunction) bool { ast.GE, ast.LE, ast.EQ, ast.NE, ast.LT, ast.GT, ast.In, ast.IsNull, ast.Like, ast.Plus, ast.Minus, ast.Div, ast.Mul, /*ast.Mod,*/ ast.If, ast.Ifnull, ast.Case, - ast.Month, - ast.TimestampDiff, ast.DateFormat, ast.FromUnixTime, + ast.Concat, ast.ConcatWS, + ast.Year, ast.Month, ast.Day, + ast.DateDiff, ast.TimestampDiff, ast.DateFormat, ast.FromUnixTime, ast.JSONLength: return true case ast.Substr, ast.Substring: @@ -1018,14 +1019,24 @@ func scalarExprSupportedByFlash(function *ScalarFunction) bool { tipb.ScalarFuncSig_CastRealAsInt, tipb.ScalarFuncSig_CastRealAsDecimal, tipb.ScalarFuncSig_CastRealAsString, tipb.ScalarFuncSig_CastRealAsTime, tipb.ScalarFuncSig_CastStringAsInt, tipb.ScalarFuncSig_CastStringAsDecimal, tipb.ScalarFuncSig_CastStringAsString, tipb.ScalarFuncSig_CastStringAsTime, tipb.ScalarFuncSig_CastStringAsReal, tipb.ScalarFuncSig_CastDecimalAsInt, tipb.ScalarFuncSig_CastDecimalAsDecimal, tipb.ScalarFuncSig_CastDecimalAsString, tipb.ScalarFuncSig_CastDecimalAsTime, - tipb.ScalarFuncSig_CastTimeAsInt, tipb.ScalarFuncSig_CastTimeAsDecimal, tipb.ScalarFuncSig_CastTimeAsTime: + tipb.ScalarFuncSig_CastTimeAsInt, tipb.ScalarFuncSig_CastTimeAsDecimal, tipb.ScalarFuncSig_CastTimeAsString, tipb.ScalarFuncSig_CastTimeAsTime: return true } - case ast.DateAdd: + case ast.DateAdd, ast.AddDate: switch function.Function.PbCode() { case tipb.ScalarFuncSig_AddDateDatetimeInt, tipb.ScalarFuncSig_AddDateStringInt: return true } + case ast.DateSub, ast.SubDate: + switch function.Function.PbCode() { + case tipb.ScalarFuncSig_SubDateDatetimeInt, tipb.ScalarFuncSig_SubDateStringInt: + return true + } + case ast.UnixTimestamp: + switch function.Function.PbCode() { + case tipb.ScalarFuncSig_UnixTimestampInt, tipb.ScalarFuncSig_UnixTimestampDec: + return true + } case ast.Round: switch function.Function.PbCode() { case tipb.ScalarFuncSig_RoundInt, tipb.ScalarFuncSig_RoundReal: From ce8773c882c3228361d068381240967832cabb1c Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 18 Jun 2021 16:50:39 +0800 Subject: [PATCH 13/27] expression: push down left/right/abs to tiflash (#25018) (#25133) --- expression/expression.go | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/expression/expression.go b/expression/expression.go index 862a53a1290d1..fb5aa6def3bc6 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -999,16 +999,19 @@ func scalarExprSupportedByFlash(function *ScalarFunction) bool { case ast.LogicOr, ast.LogicAnd, ast.UnaryNot, ast.BitNeg, ast.Xor, ast.And, ast.Or, ast.GE, ast.LE, ast.EQ, ast.NE, ast.LT, ast.GT, ast.In, ast.IsNull, ast.Like, - ast.Plus, ast.Minus, ast.Div, ast.Mul, /*ast.Mod,*/ + ast.Plus, ast.Minus, ast.Div, ast.Mul, ast.Abs, /*ast.Mod,*/ ast.If, ast.Ifnull, ast.Case, ast.Concat, ast.ConcatWS, ast.Year, ast.Month, ast.Day, ast.DateDiff, ast.TimestampDiff, ast.DateFormat, ast.FromUnixTime, ast.JSONLength: return true - case ast.Substr, ast.Substring: + case ast.Substr, ast.Substring, ast.Left, ast.Right, ast.CharLength: switch function.Function.PbCode() { case + tipb.ScalarFuncSig_LeftUTF8, + tipb.ScalarFuncSig_RightUTF8, + tipb.ScalarFuncSig_CharLengthUTF8, tipb.ScalarFuncSig_Substring2ArgsUTF8, tipb.ScalarFuncSig_Substring3ArgsUTF8: return true From 65af9912d88900ed393f5aef2508dc5640f3d0ac Mon Sep 17 00:00:00 2001 From: Fu Zhe Date: Fri, 18 Jun 2021 17:40:38 +0800 Subject: [PATCH 14/27] expression: [cherry-pick-5.0] Support push function replace down to TiFlash (#25565) --- expression/expr_to_pb_test.go | 5 +++++ expression/expression.go | 5 +++++ 2 files changed, 10 insertions(+) diff --git a/expression/expr_to_pb_test.go b/expression/expr_to_pb_test.go index fa3713af23537..a82c0e9276629 100644 --- a/expression/expr_to_pb_test.go +++ b/expression/expr_to_pb_test.go @@ -732,6 +732,11 @@ func (s *testEvaluatorSuite) TestExprPushDownToFlash(c *C) { c.Assert(err, IsNil) exprs = append(exprs, function) + // Replace + function, err = NewFunction(mock.NewContext(), ast.Replace, types.NewFieldType(mysql.TypeString), stringColumn, stringColumn, stringColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + // Substring2ArgsUTF8 function, err = NewFunction(mock.NewContext(), ast.Substr, types.NewFieldType(mysql.TypeString), stringColumn, intColumn) c.Assert(err, IsNil) diff --git a/expression/expression.go b/expression/expression.go index fb5aa6def3bc6..d591b0299a275 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -1050,6 +1050,11 @@ func scalarExprSupportedByFlash(function *ScalarFunction) bool { case tipb.ScalarFuncSig_ExtractDatetime: return true } + case ast.Replace: + switch function.Function.PbCode() { + case tipb.ScalarFuncSig_Replace: + return true + } case ast.StrToDate: switch function.Function.PbCode() { case From 59e2f486e453930dde8fe5bb36e891b3403c365e Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 18 Jun 2021 19:58:38 +0800 Subject: [PATCH 15/27] store/copr: invalidate stale regions for Mpp query. (#24410) (#24432) --- store/copr/mpp.go | 5 +++++ store/tikv/region_cache.go | 5 +++++ 2 files changed, 10 insertions(+) diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 88ac883bbfb40..d1f125aed866d 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -243,6 +243,11 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *tikv.Backoffer, m.sendError(errors.New(realResp.Error.Msg)) return } + for _, retry := range realResp.RetryRegions { + id := tikv.NewRegionVerID(retry.Id, retry.RegionEpoch.ConfVer, retry.RegionEpoch.Version) + logutil.BgLogger().Info("invalid region because tiflash detected stale region", zap.String("region id", id.String())) + m.store.GetRegionCache().InvalidateCachedRegionWithReason(id, tikv.EpochNotMatch) + } failpoint.Inject("mppNonRootTaskError", func(val failpoint.Value) { if val.(bool) && !req.IsRoot { time.Sleep(1 * time.Second) diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index ca0b47f1e5cd0..657d8f5bc9665 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -1617,6 +1617,11 @@ type RegionVerID struct { ver uint64 } +// NewRegionVerID creates a region ver id, which used for invalidating regions. +func NewRegionVerID(id, confVer, ver uint64) RegionVerID { + return RegionVerID{id, confVer, ver} +} + // GetID returns the id of the region func (r *RegionVerID) GetID() uint64 { return r.id From ad3c114bc4d2c311bb7c346b665283a37b5f733a Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 18 Jun 2021 22:04:38 +0800 Subject: [PATCH 16/27] planner/core: change agg cost factor (#25210) (#25241) --- planner/core/integration_test.go | 74 ----- planner/core/physical_plans.go | 12 +- planner/core/task.go | 18 +- .../testdata/integration_serial_suite_in.json | 20 -- .../integration_serial_suite_out.json | 291 +++--------------- planner/implementation/simple_plans.go | 4 +- 6 files changed, 69 insertions(+), 350 deletions(-) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 43e08b735a6a8..8e21dfda05f26 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -654,80 +654,6 @@ func (s *testIntegrationSerialSuite) TestMPPShuffledJoin(c *C) { } } -func (s *testIntegrationSerialSuite) TestBroadcastJoin(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("set session tidb_allow_mpp = OFF") - tk.MustExec("drop table if exists d1_t") - tk.MustExec("create table d1_t(d1_k int, value int)") - tk.MustExec("insert into d1_t values(1,2),(2,3)") - tk.MustExec("analyze table d1_t") - tk.MustExec("drop table if exists d2_t") - tk.MustExec("create table d2_t(d2_k decimal(10,2), value int)") - tk.MustExec("insert into d2_t values(10.11,2),(10.12,3)") - tk.MustExec("analyze table d2_t") - tk.MustExec("drop table if exists d3_t") - tk.MustExec("create table d3_t(d3_k date, value int)") - tk.MustExec("insert into d3_t values(date'2010-01-01',2),(date'2010-01-02',3)") - tk.MustExec("analyze table d3_t") - tk.MustExec("drop table if exists fact_t") - tk.MustExec("create table fact_t(d1_k int, d2_k decimal(10,2), d3_k date, col1 int, col2 int, col3 int)") - tk.MustExec("insert into fact_t values(1,10.11,date'2010-01-01',1,2,3),(1,10.11,date'2010-01-02',1,2,3),(1,10.12,date'2010-01-01',1,2,3),(1,10.12,date'2010-01-02',1,2,3)") - tk.MustExec("insert into fact_t values(2,10.11,date'2010-01-01',1,2,3),(2,10.11,date'2010-01-02',1,2,3),(2,10.12,date'2010-01-01',1,2,3),(2,10.12,date'2010-01-02',1,2,3)") - tk.MustExec("analyze table fact_t") - - // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) - for _, tblInfo := range db.Tables { - if tblInfo.Name.L == "fact_t" || tblInfo.Name.L == "d1_t" || tblInfo.Name.L == "d2_t" || tblInfo.Name.L == "d3_t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") - tk.MustExec("set @@session.tidb_allow_batch_cop = 1") - tk.MustExec("set @@session.tidb_opt_broadcast_join = 1") - // make cbo force choose broadcast join since sql hint does not work for semi/anti-semi join - tk.MustExec("set @@session.tidb_opt_cpu_factor=10000000;") - var input []string - var output []struct { - SQL string - Plan []string - } - s.testData.GetTestCases(c, &input, &output) - for i, tt := range input { - s.testData.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } - - // out table of out join should not be global - _, err := tk.Exec("explain format = 'brief' select /*+ broadcast_join(fact_t, d1_t), broadcast_join_local(d1_t) */ count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[planner:1815]Internal : Can't find a proper physical plan for this query") - // nullEQ not supported - _, err = tk.Exec("explain format = 'brief' select /*+ broadcast_join(fact_t, d1_t) */ count(*) from fact_t join d1_t on fact_t.d1_k <=> d1_t.d1_k") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[planner:1815]Internal : Can't find a proper physical plan for this query") - // not supported if join condition has unsupported expr - _, err = tk.Exec("explain format = 'brief' select /*+ broadcast_join(fact_t, d1_t) */ count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and sqrt(fact_t.col1) > 2") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[planner:1815]Internal : Can't find a proper physical plan for this query") - // cartsian join not supported - _, err = tk.Exec("explain format = 'brief' select /*+ broadcast_join(fact_t, d1_t) */ count(*) from fact_t join d1_t") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[planner:1815]Internal : Can't find a proper physical plan for this query") -} - func (s *testIntegrationSerialSuite) TestJoinNotSupportedByTiFlash(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 43304971b4680..a783d4f3d51db 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -1008,7 +1008,7 @@ func (p *basePhysicalAgg) numDistinctFunc() (num int) { return } -func (p *basePhysicalAgg) getAggFuncCostFactor() (factor float64) { +func (p *basePhysicalAgg) getAggFuncCostFactor(isMPP bool) (factor float64) { factor = 0.0 for _, agg := range p.AggFuncs { if fac, ok := aggFuncFactor[agg.Name]; ok { @@ -1018,7 +1018,15 @@ func (p *basePhysicalAgg) getAggFuncCostFactor() (factor float64) { } } if factor == 0 { - factor = 1.0 + if isMPP { + // The default factor 1.0 will lead to 1-phase agg in pseudo stats settings. + // But in mpp cases, 2-phase is more usual. So we change this factor. + // TODO: This is still a little tricky and might cause regression. We should + // calibrate these factors and polish our cost model in the future. + factor = aggFuncFactor[ast.AggFuncFirstRow] + } else { + factor = 1.0 + } } return } diff --git a/planner/core/task.go b/planner/core/task.go index 06cfcb1aa2878..0138a0943231b 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -1739,7 +1739,7 @@ func (p *PhysicalStreamAgg) attach2Task(tasks ...task) task { // GetCost computes cost of stream aggregation considering CPU/memory. func (p *PhysicalStreamAgg) GetCost(inputRows float64, isRoot bool) float64 { - aggFuncFactor := p.getAggFuncCostFactor() + aggFuncFactor := p.getAggFuncCostFactor(false) var cpuCost float64 sessVars := p.ctx.GetSessionVars() if isRoot { @@ -1786,7 +1786,7 @@ func (p *PhysicalHashAgg) attach2TaskForMpp(tasks ...task) task { if proj != nil { attachPlan2Task(proj, mpp) } - mpp.addCost(p.GetCost(inputRows, false)) + mpp.addCost(p.GetCost(inputRows, false, true)) return mpp case Mpp2Phase: proj := p.convertAvgForMPP() @@ -1817,18 +1817,18 @@ func (p *PhysicalHashAgg) attach2TaskForMpp(tasks ...task) task { attachPlan2Task(proj, newMpp) } // TODO: how to set 2-phase cost? - newMpp.addCost(p.GetCost(inputRows, false)) + newMpp.addCost(p.GetCost(inputRows, false, true)) return newMpp case MppTiDB: partialAgg, finalAgg := p.newPartialAggregate(kv.TiFlash, false) if partialAgg != nil { attachPlan2Task(partialAgg, mpp) } - mpp.addCost(p.GetCost(inputRows, false)) + mpp.addCost(p.GetCost(inputRows, false, true)) t = mpp.convertToRootTask(p.ctx) inputRows = t.count() attachPlan2Task(finalAgg, t) - t.addCost(p.GetCost(inputRows, true)) + t.addCost(p.GetCost(inputRows, true, false)) return t default: return invalidTask @@ -1858,7 +1858,7 @@ func (p *PhysicalHashAgg) attach2Task(tasks ...task) task { partialAgg.SetChildren(cop.indexPlan) cop.indexPlan = partialAgg } - cop.addCost(p.GetCost(inputRows, false)) + cop.addCost(p.GetCost(inputRows, false, false)) } // In `newPartialAggregate`, we are using stats of final aggregation as stats // of `partialAgg`, so the network cost of transferring result rows of `partialAgg` @@ -1891,15 +1891,15 @@ func (p *PhysicalHashAgg) attach2Task(tasks ...task) task { // hash aggregation, it would cause under-estimation as the reason mentioned in comment above. // To make it simple, we also treat 2-phase parallel hash aggregation in TiDB layer as // 1-phase when computing cost. - t.addCost(p.GetCost(inputRows, true)) + t.addCost(p.GetCost(inputRows, true, false)) return t } // GetCost computes the cost of hash aggregation considering CPU/memory. -func (p *PhysicalHashAgg) GetCost(inputRows float64, isRoot bool) float64 { +func (p *PhysicalHashAgg) GetCost(inputRows float64, isRoot bool, isMPP bool) float64 { cardinality := p.statsInfo().RowCount numDistinctFunc := p.numDistinctFunc() - aggFuncFactor := p.getAggFuncCostFactor() + aggFuncFactor := p.getAggFuncCostFactor(isMPP) var cpuCost float64 sessVars := p.ctx.GetSessionVars() if isRoot { diff --git a/planner/core/testdata/integration_serial_suite_in.json b/planner/core/testdata/integration_serial_suite_in.json index 30bf481a15aca..8733b9a97e8c2 100644 --- a/planner/core/testdata/integration_serial_suite_in.json +++ b/planner/core/testdata/integration_serial_suite_in.json @@ -80,26 +80,6 @@ "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)" ] }, - { - "name": "TestBroadcastJoin", - "cases": [ - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t,d2_t,d3_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t), broadcast_join_local(d1_t) */ count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t,d2_t,d3_t), broadcast_join_local(d2_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > d1_t.value", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > 10", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col2 > 10 and fact_t.col1 > d1_t.value", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10 and fact_t.col1 > d1_t.value", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k)", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k)", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)" - ] - }, { "name": "TestJoinNotSupportedByTiFlash", "cases": [ diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index 2d03c51d31e7c..20ab278ed0cb2 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -768,208 +768,6 @@ } ] }, - { - "Name": "TestBroadcastJoin", - "Cases": [ - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false, global read", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t,d2_t,d3_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#18)->Column#17", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#18", - " └─HashJoin 8.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d3_k, test.d3_t.d3_k)]", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d3_t.d3_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d3_t keep order:false, global read", - " └─HashJoin(Probe) 8.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d2_k, test.d2_t.d2_k)]", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d2_t.d2_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d2_t keep order:false, global read", - " └─HashJoin(Probe) 8.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false, global read", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t), broadcast_join_local(d1_t) */ count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false, global read" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t,d2_t,d3_t), broadcast_join_local(d2_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#18)->Column#17", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#18", - " └─HashJoin 8.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d3_k, test.d3_t.d3_k)]", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d3_t.d3_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d3_t keep order:false, global read", - " └─HashJoin(Probe) 8.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d2_k, test.d2_t.d2_k)]", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d2_t.d2_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d2_t keep order:false", - " └─HashJoin(Probe) 8.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false, global read", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false, global read" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 batchCop[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false, global read", - " └─TableFullScan(Probe) 8.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 batchCop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─TableFullScan(Build) 2.00 batchCop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false, global read" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > d1_t.value", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.fact_t.col1, test.d1_t.value)", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false, global read", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > 10", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 batchCop[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], left cond:[gt(test.fact_t.col1, 10)]", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false, global read", - " └─TableFullScan(Probe) 8.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col2 > 10 and fact_t.col1 > d1_t.value", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 batchCop[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], left cond:[gt(test.fact_t.col2, 10)], other cond:gt(test.fact_t.col1, test.d1_t.value)", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false, global read", - " └─TableFullScan(Probe) 8.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 batchCop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], right cond:gt(test.d1_t.value, 10)", - " ├─TableFullScan(Build) 2.00 batchCop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false, global read" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10 and fact_t.col1 > d1_t.value", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 batchCop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], right cond:gt(test.d1_t.value, 10), other cond:gt(test.fact_t.col1, test.d1_t.value)", - " ├─Selection(Build) 8.00 batchCop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false, global read", - " └─TableFullScan(Probe) 2.00 batchCop[tiflash] table:d1_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k)", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 6.40 batchCop[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false, global read", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 6.40 batchCop[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false, global read", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k)", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 6.40 batchCop[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─TableFullScan(Build) 2.00 batchCop[tiflash] table:d1_t keep order:false, global read", - " └─TableFullScan(Probe) 8.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 6.40 batchCop[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", - " ├─TableFullScan(Build) 2.00 batchCop[tiflash] table:d1_t keep order:false, global read", - " └─TableFullScan(Probe) 8.00 batchCop[tiflash] table:fact_t keep order:false" - ] - } - ] - }, { "Name": "TestJoinNotSupportedByTiFlash", "Cases": [ @@ -1947,23 +1745,24 @@ "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", " └─Projection 8000.00 batchCop[tiflash] test.tt.col_varchar_64, test.tt.col_char_64_not_null", " └─HashAgg 8000.00 batchCop[tiflash] group by:test.tt.col_char_64_not_null, test.tt.col_varchar_64, funcs:firstrow(test.tt.col_varchar_64)->test.tt.col_varchar_64, funcs:firstrow(test.tt.col_char_64_not_null)->test.tt.col_char_64_not_null", - " └─ExchangeReceiver 15609.38 batchCop[tiflash] ", - " └─ExchangeSender 15609.38 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.tt.col_varchar_64, test.tt.col_char_64_not_null", - " └─HashJoin 15609.38 batchCop[tiflash] inner join, equal:[eq(test.tt.col_char_64_not_null, test.tt.col_varchar_64)]", - " ├─ExchangeReceiver(Build) 10000.00 batchCop[tiflash] ", - " │ └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#29", - " │ └─Projection 10000.00 batchCop[tiflash] test.tt.col_varchar_64, test.tt.col_char_64_not_null, cast(test.tt.col_char_64_not_null, varchar(64) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#29", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo", - " └─HashJoin(Probe) 12487.50 batchCop[tiflash] inner join, equal:[eq(test.tt.col_varchar_key, test.tt.col_varchar_64) eq(Column#19, test.tt.col_decimal_30_10_key)]", - " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.tt.col_varchar_key", - " │ └─Projection 9990.00 batchCop[tiflash] test.tt.col_varchar_key, cast(test.tt.col_tinyint, decimal(20,0) BINARY)->Column#19", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.tt.col_varchar_key))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t2 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", - " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.tt.col_varchar_64", - " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.tt.col_varchar_64))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t3 keep order:false, stats:pseudo" + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.tt.col_varchar_64, test.tt.col_char_64_not_null", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.tt.col_char_64_not_null, test.tt.col_varchar_64, ", + " └─HashJoin 15609.38 batchCop[tiflash] inner join, equal:[eq(test.tt.col_char_64_not_null, test.tt.col_varchar_64)]", + " ├─ExchangeReceiver(Build) 10000.00 batchCop[tiflash] ", + " │ └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#30", + " │ └─Projection 10000.00 batchCop[tiflash] test.tt.col_varchar_64, test.tt.col_char_64_not_null, cast(test.tt.col_char_64_not_null, varchar(64) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#30", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 batchCop[tiflash] inner join, equal:[eq(test.tt.col_varchar_key, test.tt.col_varchar_64) eq(Column#19, test.tt.col_decimal_30_10_key)]", + " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", + " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.tt.col_varchar_key", + " │ └─Projection 9990.00 batchCop[tiflash] test.tt.col_varchar_key, cast(test.tt.col_tinyint, decimal(20,0) BINARY)->Column#19", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.tt.col_varchar_key))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t2 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", + " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.tt.col_varchar_64", + " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.tt.col_varchar_64))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t3 keep order:false, stats:pseudo" ] } ] @@ -2173,20 +1972,22 @@ " └─HashJoin 7992.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", " ├─Projection(Build) 7992.00 batchCop[tiflash] test.t.id", " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, ", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", " └─Projection(Probe) 7992.00 batchCop[tiflash] Column#11, test.t.id", " └─HashAgg 7992.00 batchCop[tiflash] group by:Column#32, funcs:sum(Column#30)->Column#11, funcs:firstrow(Column#31)->test.t.id", " └─Projection 9990.00 batchCop[tiflash] cast(test.t.id, decimal(32,0) BINARY)->Column#30, test.t.id, test.t.id", " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", " ├─Projection(Build) 7992.00 batchCop[tiflash] test.t.id", " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, ", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id", " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", @@ -2263,10 +2064,11 @@ " └─Projection 6400.00 batchCop[tiflash] cast(test.t.id, decimal(32,0) BINARY)->Column#21, test.t.value", " └─Projection 6400.00 batchCop[tiflash] test.t.id, test.t.value", " └─HashAgg 6400.00 batchCop[tiflash] group by:test.t.id, test.t.value, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.value)->test.t.value", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.value", - " └─Selection 8000.00 batchCop[tiflash] gt(cast(test.t.id), test.t.value)", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeReceiver 6400.00 batchCop[tiflash] ", + " └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.value", + " └─HashAgg 6400.00 batchCop[tiflash] group by:test.t.id, test.t.value, ", + " └─Selection 8000.00 batchCop[tiflash] gt(cast(test.t.id), test.t.value)", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -2401,10 +2203,11 @@ " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: Broadcast", " │ └─Projection 7992.00 batchCop[tiflash] test.t.id", " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, ", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", " └─Projection(Probe) 7992.00 batchCop[tiflash] Column#7, test.t.id", " └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#9)->Column#7, funcs:firstrow(test.t.id)->test.t.id", " └─ExchangeReceiver 7992.00 batchCop[tiflash] ", @@ -2424,10 +2227,11 @@ " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: Broadcast", " │ └─Projection 7992.00 batchCop[tiflash] test.t.id", " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, ", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", " └─Projection(Probe) 7992.00 batchCop[tiflash] Column#11, test.t.id", " └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#14)->Column#11, funcs:firstrow(test.t.id)->test.t.id", " └─ExchangeReceiver 7992.00 batchCop[tiflash] ", @@ -2439,10 +2243,11 @@ " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: Broadcast", " │ └─Projection 7992.00 batchCop[tiflash] test.t.id", " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, ", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] diff --git a/planner/implementation/simple_plans.go b/planner/implementation/simple_plans.go index cb49fd0e10225..0a727576b3950 100644 --- a/planner/implementation/simple_plans.go +++ b/planner/implementation/simple_plans.go @@ -85,7 +85,7 @@ type TiDBHashAggImpl struct { // CalcCost implements Implementation CalcCost interface. func (agg *TiDBHashAggImpl) CalcCost(outCount float64, children ...memo.Implementation) float64 { hashAgg := agg.plan.(*plannercore.PhysicalHashAgg) - selfCost := hashAgg.GetCost(children[0].GetPlan().Stats().RowCount, true) + selfCost := hashAgg.GetCost(children[0].GetPlan().Stats().RowCount, true, false) agg.cost = selfCost + children[0].GetCost() return agg.cost } @@ -110,7 +110,7 @@ type TiKVHashAggImpl struct { // CalcCost implements Implementation CalcCost interface. func (agg *TiKVHashAggImpl) CalcCost(outCount float64, children ...memo.Implementation) float64 { hashAgg := agg.plan.(*plannercore.PhysicalHashAgg) - selfCost := hashAgg.GetCost(children[0].GetPlan().Stats().RowCount, false) + selfCost := hashAgg.GetCost(children[0].GetPlan().Stats().RowCount, false, false) agg.cost = selfCost + children[0].GetCost() return agg.cost } From 772a905531c3fdb4ec9a02e2fc0d9a4347dd4ffb Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 18 Jun 2021 22:16:38 +0800 Subject: [PATCH 17/27] store/tikv: change backoff type for missed tiflash peer. (#24577) (#24600) --- store/copr/batch_coprocessor.go | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 04f30bc0e8a88..43b77b06919d8 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -272,8 +272,10 @@ func buildBatchCopTasks(bo *tikv.Backoffer, cache *tikv.RegionCache, ranges *tik if err != nil { return nil, errors.Trace(err) } - // If the region is not found in cache, it must be out - // of date and already be cleaned up. We should retry and generate new tasks. + // When rpcCtx is nil, it's not only attributed to the miss region, but also + // some TiFlash stores crash and can't be recovered. + // That is not an error that can be easily recovered, so we regard this error + // same as rpc error. if rpcCtx == nil { needRetry = true logutil.BgLogger().Info("retry for TiFlash peer with region missing", zap.Uint64("region id", task.region.GetID())) @@ -295,8 +297,10 @@ func buildBatchCopTasks(bo *tikv.Backoffer, cache *tikv.RegionCache, ranges *tik } } if needRetry { - // Backoff once for each retry. - err = bo.Backoff(tikv.BoRegionMiss, errors.New("Cannot find region with TiFlash peer")) + // As mentioned above, nil rpcCtx is always attributed to failed stores. + // It's equal to long poll the store but get no response. Here we'd better use + // TiFlash error to trigger the TiKV fallback mechanism. + err = bo.Backoff(tikv.BoTiFlashRPC, errors.New("Cannot find region with TiFlash peer")) if err != nil { return nil, errors.Trace(err) } From 7ef7ae20b8efb36c3186931ea9c5781bb1fa106b Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Mon, 21 Jun 2021 12:00:52 +0800 Subject: [PATCH 18/27] ranger: fix the case which could have duplicate ranges (#24590) (#24635) --- util/ranger/ranger.go | 6 +++--- util/ranger/ranger_test.go | 10 +++++++++- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/util/ranger/ranger.go b/util/ranger/ranger.go index a414fb7fec75a..15617888da40e 100644 --- a/util/ranger/ranger.go +++ b/util/ranger/ranger.go @@ -464,7 +464,7 @@ func fixPrefixColRange(ranges []*Range, lengths []int, tp []*types.FieldType) bo for _, ran := range ranges { lowTail := len(ran.LowVal) - 1 for i := 0; i < lowTail; i++ { - CutDatumByPrefixLen(&ran.LowVal[i], lengths[i], tp[i]) + hasCut = CutDatumByPrefixLen(&ran.LowVal[i], lengths[i], tp[i]) || hasCut } lowCut := CutDatumByPrefixLen(&ran.LowVal[lowTail], lengths[lowTail], tp[lowTail]) // If the length of the last column of LowVal is equal to the prefix length, LowExclude should be set false. @@ -475,13 +475,13 @@ func fixPrefixColRange(ranges []*Range, lengths []int, tp []*types.FieldType) bo } highTail := len(ran.HighVal) - 1 for i := 0; i < highTail; i++ { - CutDatumByPrefixLen(&ran.HighVal[i], lengths[i], tp[i]) + hasCut = CutDatumByPrefixLen(&ran.HighVal[i], lengths[i], tp[i]) || hasCut } highCut := CutDatumByPrefixLen(&ran.HighVal[highTail], lengths[highTail], tp[highTail]) if highCut { ran.HighExclude = false } - hasCut = lowCut || highCut + hasCut = hasCut || lowCut || highCut } return hasCut } diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index 4038e197f8721..57137370bf626 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -351,7 +351,8 @@ create table t( index idx_cb(c, a), index idx_d(d(2)), index idx_e(e(2)), - index idx_f(f) + index idx_f(f), + index idx_de(d(2), e) )`) tests := []struct { @@ -620,6 +621,13 @@ create table t( filterConds: "[like(test.t.f, @%, 92)]", resultStr: "[[NULL,+inf]]", }, + { + indexPos: 5, + exprStr: "d in ('aab', 'aac') and e = 'a'", + accessConds: "[in(test.t.d, aab, aac) eq(test.t.e, a)]", + filterConds: "[in(test.t.d, aab, aac)]", + resultStr: "[[\"aa\" 0x61,\"aa\" 0x61]]", + }, } collate.SetNewCollationEnabledForTest(true) From 8fa36869fc54cc99df48525ad79df6d3f451d298 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Mon, 21 Jun 2021 17:14:54 +0800 Subject: [PATCH 19/27] planner/core: support union all for mpp. (#24287) (#25051) --- distsql/select_result.go | 52 +++++-- executor/mpp_gather.go | 24 ++-- executor/tiflash_test.go | 54 +++++++ expression/column.go | 3 +- planner/core/exhaust_physical_plans.go | 34 ++++- planner/core/fragment.go | 139 ++++++++++++++++--- planner/core/initialize.go | 2 +- planner/core/logical_plan_builder.go | 4 + planner/core/physical_plans.go | 45 +++++- planner/core/plan.go | 3 + planner/core/rule_inject_extra_projection.go | 34 +++++ planner/core/task.go | 23 +++ store/copr/mpp.go | 1 + util/execdetails/execdetails.go | 3 +- 14 files changed, 369 insertions(+), 52 deletions(-) diff --git a/distsql/select_result.go b/distsql/select_result.go index 9e37a02796f4f..05d7c41cf157b 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -295,13 +295,6 @@ func (r *selectResult) updateCopRuntimeStats(ctx context.Context, copStats *copr if r.rootPlanID <= 0 || r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl == nil || callee == "" { return } - if len(r.selectResp.GetExecutionSummaries()) != len(r.copPlanIDs) { - logutil.Logger(ctx).Error("invalid cop task execution summaries length", - zap.Int("expected", len(r.copPlanIDs)), - zap.Int("received", len(r.selectResp.GetExecutionSummaries()))) - - return - } if r.stats == nil { id := r.rootPlanID r.stats = &selectResultRuntimeStats{ @@ -316,12 +309,49 @@ func (r *selectResult) updateCopRuntimeStats(ctx context.Context, copStats *copr r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RecordScanDetail(r.copPlanIDs[len(r.copPlanIDs)-1], r.storeType.Name(), copStats.ScanDetail) } - for i, detail := range r.selectResp.GetExecutionSummaries() { + // If hasExecutor is true, it means the summary is returned from TiFlash. + hasExecutor := false + for _, detail := range r.selectResp.GetExecutionSummaries() { if detail != nil && detail.TimeProcessedNs != nil && detail.NumProducedRows != nil && detail.NumIterations != nil { - planID := r.copPlanIDs[i] - r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl. - RecordOneCopTask(planID, r.storeType.Name(), callee, detail) + if detail.ExecutorId != nil { + hasExecutor = true + } + break + } + } + if hasExecutor { + var recorededPlanIDs = make(map[int]int) + for i, detail := range r.selectResp.GetExecutionSummaries() { + if detail != nil && detail.TimeProcessedNs != nil && + detail.NumProducedRows != nil && detail.NumIterations != nil { + planID := r.copPlanIDs[i] + recorededPlanIDs[r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl. + RecordOneCopTask(planID, r.storeType.Name(), callee, detail)] = 0 + } + } + num := uint64(0) + dummySummary := &tipb.ExecutorExecutionSummary{TimeProcessedNs: &num, NumProducedRows: &num, NumIterations: &num, ExecutorId: nil} + for _, planID := range r.copPlanIDs { + if _, ok := recorededPlanIDs[planID]; !ok { + r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RecordOneCopTask(planID, r.storeType.Name(), callee, dummySummary) + } + } + } else { + // For cop task cases, we still need this protection. + if len(r.selectResp.GetExecutionSummaries()) != len(r.copPlanIDs) { + logutil.Logger(ctx).Error("invalid cop task execution summaries length", + zap.Int("expected", len(r.copPlanIDs)), + zap.Int("received", len(r.selectResp.GetExecutionSummaries()))) + return + } + for i, detail := range r.selectResp.GetExecutionSummaries() { + if detail != nil && detail.TimeProcessedNs != nil && + detail.NumProducedRows != nil && detail.NumIterations != nil { + planID := r.copPlanIDs[i] + r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl. + RecordOneCopTask(planID, r.storeType.Name(), callee, detail) + } } } } diff --git a/executor/mpp_gather.go b/executor/mpp_gather.go index d34d63be49a79..536a06eda8993 100644 --- a/executor/mpp_gather.go +++ b/executor/mpp_gather.go @@ -50,7 +50,7 @@ type MPPGather struct { respIter distsql.SelectResult } -func (e *MPPGather) appendMPPDispatchReq(pf *plannercore.Fragment, tasks []*kv.MPPTask, isRoot bool) error { +func (e *MPPGather) appendMPPDispatchReq(pf *plannercore.Fragment) error { dagReq, _, err := constructDAGReq(e.ctx, []plannercore.PhysicalPlan{pf.ExchangeSender}, kv.TiFlash) if err != nil { return errors.Trace(err) @@ -58,12 +58,12 @@ func (e *MPPGather) appendMPPDispatchReq(pf *plannercore.Fragment, tasks []*kv.M for i := range pf.ExchangeSender.Schema().Columns { dagReq.OutputOffsets = append(dagReq.OutputOffsets, uint32(i)) } - if !isRoot { + if !pf.IsRoot { dagReq.EncodeType = tipb.EncodeType_TypeCHBlock } else { dagReq.EncodeType = tipb.EncodeType_TypeChunk } - for _, mppTask := range tasks { + for _, mppTask := range pf.ExchangeSender.Tasks { err := updateExecutorTableID(context.Background(), dagReq.RootExecutor, mppTask.TableID, true) if err != nil { return errors.Trace(err) @@ -77,7 +77,7 @@ func (e *MPPGather) appendMPPDispatchReq(pf *plannercore.Fragment, tasks []*kv.M Data: pbData, Meta: mppTask.Meta, ID: mppTask.ID, - IsRoot: isRoot, + IsRoot: pf.IsRoot, Timeout: 10, SchemaVar: e.is.SchemaMetaVersion(), StartTs: e.startTS, @@ -85,12 +85,6 @@ func (e *MPPGather) appendMPPDispatchReq(pf *plannercore.Fragment, tasks []*kv.M } e.mppReqs = append(e.mppReqs, req) } - for _, r := range pf.ExchangeReceivers { - err = e.appendMPPDispatchReq(r.GetExchangeSender().Fragment, r.Tasks, false) - if err != nil { - return errors.Trace(err) - } - } return nil } @@ -108,13 +102,15 @@ func (e *MPPGather) Open(ctx context.Context) (err error) { // TODO: Move the construct tasks logic to planner, so we can see the explain results. sender := e.originalPlan.(*plannercore.PhysicalExchangeSender) planIDs := collectPlanIDS(e.originalPlan, nil) - rootTasks, err := plannercore.GenerateRootMPPTasks(e.ctx, e.startTS, sender, e.is) + frags, err := plannercore.GenerateRootMPPTasks(e.ctx, e.startTS, sender, e.is) if err != nil { return errors.Trace(err) } - err = e.appendMPPDispatchReq(sender.Fragment, rootTasks, true) - if err != nil { - return errors.Trace(err) + for _, frag := range frags { + err = e.appendMPPDispatchReq(frag) + if err != nil { + return errors.Trace(err) + } } failpoint.Inject("checkTotalMPPTasks", func(val failpoint.Value) { if val.(int) != len(e.mppReqs) { diff --git a/executor/tiflash_test.go b/executor/tiflash_test.go index 0c2763d299148..2727bdc15fb56 100644 --- a/executor/tiflash_test.go +++ b/executor/tiflash_test.go @@ -447,6 +447,60 @@ func (s *tiflashTestSuite) TestMppGoroutinesExitFromErrors(c *C) { c.Assert(failpoint.Disable(hang), IsNil) } +func (s *tiflashTestSuite) TestMppUnionAll(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists x1") + tk.MustExec("create table x1(a int , b int);") + tk.MustExec("alter table x1 set tiflash replica 1") + tk.MustExec("drop table if exists x2") + tk.MustExec("create table x2(a int , b int);") + tk.MustExec("alter table x2 set tiflash replica 1") + tb := testGetTableByName(c, tk.Se, "test", "x1") + err := domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, tb.Meta().ID, true) + c.Assert(err, IsNil) + tb = testGetTableByName(c, tk.Se, "test", "x2") + err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, tb.Meta().ID, true) + c.Assert(err, IsNil) + + tk.MustExec("set @@session.tidb_opt_mpp_outer_join_fixed_build_side=1") + + tk.MustExec("insert into x1 values (1, 1), (2, 2), (3, 3), (4, 4)") + tk.MustExec("insert into x2 values (5, 1), (2, 2), (3, 3), (4, 4)") + + // test join + union (join + select) + tk.MustQuery("select x1.a, x.a from x1 left join (select x2.b a, x1.b from x1 join x2 on x1.a = x2.b union all select * from x1 ) x on x1.a = x.a order by x1.a").Check(testkit.Rows("1 1", "1 1", "2 2", "2 2", "3 3", "3 3", "4 4", "4 4")) + tk.MustQuery("select x1.a, x.a from x1 left join (select count(*) a, sum(b) b from x1 group by a union all select * from x2 ) x on x1.a = x.a order by x1.a").Check(testkit.Rows("1 1", "1 1", "1 1", "1 1", "2 2", "3 3", "4 4")) + + tk.MustExec("drop table if exists x3") + tk.MustExec("create table x3(a int , b int);") + tk.MustExec("alter table x3 set tiflash replica 1") + tb = testGetTableByName(c, tk.Se, "test", "x3") + err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, tb.Meta().ID, true) + c.Assert(err, IsNil) + + tk.MustExec("insert into x3 values (2, 2), (2, 3), (2, 4)") + // test nested union all + tk.MustQuery("select count(*) from (select a, b from x1 union all select a, b from x3 union all (select x1.a, x3.b from (select * from x3 union all select * from x2) x3 left join x1 on x3.a = x1.b))").Check(testkit.Rows("14")) + // test union all join union all + tk.MustQuery("select count(*) from (select * from x1 union all select * from x2 union all select * from x3) x join (select * from x1 union all select * from x2 union all select * from x3) y on x.a = y.b").Check(testkit.Rows("29")) + tk.MustExec("set @@session.tidb_broadcast_join_threshold_count=100000") + failpoint.Enable("github.com/pingcap/tidb/executor/checkTotalMPPTasks", `return(6)`) + tk.MustQuery("select count(*) from (select * from x1 union all select * from x2 union all select * from x3) x join (select * from x1 union all select * from x2 union all select * from x3) y on x.a = y.b").Check(testkit.Rows("29")) + failpoint.Disable("github.com/pingcap/tidb/executor/checkTotalMPPTasks") + + tk.MustExec("drop table if exists x4") + tk.MustExec("create table x4(a int not null, b int not null);") + tk.MustExec("alter table x4 set tiflash replica 1") + tb = testGetTableByName(c, tk.Se, "test", "x4") + err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, tb.Meta().ID, true) + c.Assert(err, IsNil) + + tk.MustExec("insert into x4 values (2, 2), (2, 3)") + tk.MustQuery("(select * from x1 union all select * from x4) order by a, b").Check(testkit.Rows("1 1", "2 2", "2 2", "2 3", "3 3", "4 4")) + +} + func (s *tiflashTestSuite) TestMppApply(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/expression/column.go b/expression/column.go index 006b9a3867cda..ebc0feaf93a06 100644 --- a/expression/column.go +++ b/expression/column.go @@ -38,10 +38,9 @@ type CorrelatedColumn struct { // Clone implements Expression interface. func (col *CorrelatedColumn) Clone() Expression { - var d types.Datum return &CorrelatedColumn{ Column: col.Column, - Data: &d, + Data: col.Data, } } diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index b3f10d6bfabc7..c420ac77d4afa 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -2151,7 +2151,7 @@ func (p *baseLogicalPlan) canPushToCop(storeTp kv.StoreType) bool { } } ret = ret && validDs - case *LogicalAggregation, *LogicalProjection, *LogicalSelection, *LogicalJoin: + case *LogicalAggregation, *LogicalProjection, *LogicalSelection, *LogicalJoin, *LogicalUnionAll: if storeTp == kv.TiFlash { ret = ret && c.canPushToCop(storeTp) } else { @@ -2519,15 +2519,41 @@ func (p *LogicalLock) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P func (p *LogicalUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { // TODO: UnionAll can not pass any order, but we can change it to sort merge to keep order. - if !prop.IsEmpty() || prop.IsFlashProp() { + if !prop.IsEmpty() || (prop.IsFlashProp() && prop.TaskTp != property.MppTaskType) { + return nil, true + } + // TODO: UnionAll can pass partition info, but for briefness, we prevent it from pushing down. + if prop.TaskTp == property.MppTaskType && prop.PartitionTp != property.AnyType { return nil, true } + canUseMpp := p.ctx.GetSessionVars().AllowMPPExecution && p.canPushToCop(kv.TiFlash) chReqProps := make([]*property.PhysicalProperty, 0, len(p.children)) for range p.children { - chReqProps = append(chReqProps, &property.PhysicalProperty{ExpectedCnt: prop.ExpectedCnt}) + if canUseMpp && prop.TaskTp == property.MppTaskType { + chReqProps = append(chReqProps, &property.PhysicalProperty{ + ExpectedCnt: prop.ExpectedCnt, + TaskTp: property.MppTaskType, + }) + } else { + chReqProps = append(chReqProps, &property.PhysicalProperty{ExpectedCnt: prop.ExpectedCnt}) + } } - ua := PhysicalUnionAll{}.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, chReqProps...) + ua := PhysicalUnionAll{ + mpp: canUseMpp && prop.TaskTp == property.MppTaskType, + }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, chReqProps...) ua.SetSchema(p.Schema()) + if canUseMpp && prop.TaskTp == property.RootTaskType { + chReqProps = make([]*property.PhysicalProperty, 0, len(p.children)) + for range p.children { + chReqProps = append(chReqProps, &property.PhysicalProperty{ + ExpectedCnt: prop.ExpectedCnt, + TaskTp: property.MppTaskType, + }) + } + mppUA := PhysicalUnionAll{mpp: true}.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, chReqProps...) + mppUA.SetSchema(p.Schema()) + return []PhysicalPlan{ua, mppUA}, true + } return []PhysicalPlan{ua}, true } diff --git a/planner/core/fragment.go b/planner/core/fragment.go index 911855ea22aa9..f329374d853f5 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -38,32 +38,49 @@ type Fragment struct { // following fields are filled after scheduling. ExchangeSender *PhysicalExchangeSender // data exporter + + IsRoot bool +} + +type tasksAndFrags struct { + tasks []*kv.MPPTask + frags []*Fragment } type mppTaskGenerator struct { ctx sessionctx.Context startTS uint64 is infoschema.InfoSchema + frags []*Fragment + cache map[int]tasksAndFrags } // GenerateRootMPPTasks generate all mpp tasks and return root ones. -func GenerateRootMPPTasks(ctx sessionctx.Context, startTs uint64, sender *PhysicalExchangeSender, is infoschema.InfoSchema) ([]*kv.MPPTask, error) { - g := &mppTaskGenerator{ctx: ctx, startTS: startTs, is: is} +func GenerateRootMPPTasks(ctx sessionctx.Context, startTs uint64, sender *PhysicalExchangeSender, is infoschema.InfoSchema) ([]*Fragment, error) { + g := &mppTaskGenerator{ + ctx: ctx, + startTS: startTs, + is: is, + cache: make(map[int]tasksAndFrags), + } return g.generateMPPTasks(sender) } -func (e *mppTaskGenerator) generateMPPTasks(s *PhysicalExchangeSender) ([]*kv.MPPTask, error) { +func (e *mppTaskGenerator) generateMPPTasks(s *PhysicalExchangeSender) ([]*Fragment, error) { logutil.BgLogger().Info("Mpp will generate tasks", zap.String("plan", ToString(s))) tidbTask := &kv.MPPTask{ StartTs: e.startTS, ID: -1, } - rootTasks, err := e.generateMPPTasksForFragment(s) + _, frags, err := e.generateMPPTasksForExchangeSender(s) if err != nil { return nil, errors.Trace(err) } - s.TargetTasks = []*kv.MPPTask{tidbTask} - return rootTasks, nil + for _, frag := range frags { + frag.ExchangeSender.TargetTasks = []*kv.MPPTask{tidbTask} + frag.IsRoot = true + } + return e.frags, nil } type mppAddr struct { @@ -105,6 +122,8 @@ func (f *Fragment) init(p PhysicalPlan) error { f.TableScan = x case *PhysicalExchangeReceiver: f.ExchangeReceivers = append(f.ExchangeReceivers, x) + case *PhysicalUnionAll: + return errors.New("unexpected union all detected") default: for _, ch := range p.Children() { if err := f.init(ch); err != nil { @@ -115,20 +134,107 @@ func (f *Fragment) init(p PhysicalPlan) error { return nil } -func newFragment(s *PhysicalExchangeSender) (*Fragment, error) { - f := &Fragment{ExchangeSender: s} - s.Fragment = f - err := f.init(s) - return f, errors.Trace(err) +// We would remove all the union-all operators by 'untwist'ing and copying the plans above union-all. +// This will make every route from root (ExchangeSender) to leaf nodes (ExchangeReceiver and TableScan) +// a new ioslated tree (and also a fragment) without union all. These trees (fragments then tasks) will +// finally be gathered to TiDB or be exchanged to upper tasks again. +// For instance, given a plan "select c1 from t union all select c1 from s" +// after untwist, there will be two plans in `forest` slice: +// - ExchangeSender -> Projection (c1) -> TableScan(t) +// - ExchangeSender -> Projection (c2) -> TableScan(s) +func untwistPlanAndRemoveUnionAll(stack []PhysicalPlan, forest *[]*PhysicalExchangeSender) error { + cur := stack[len(stack)-1] + switch x := cur.(type) { + case *PhysicalTableScan, *PhysicalExchangeReceiver: // This should be the leave node. + p, err := stack[0].Clone() + if err != nil { + return errors.Trace(err) + } + *forest = append(*forest, p.(*PhysicalExchangeSender)) + for i := 1; i < len(stack); i++ { + if _, ok := stack[i].(*PhysicalUnionAll); ok { + continue + } + ch, err := stack[i].Clone() + if err != nil { + return errors.Trace(err) + } + if join, ok := p.(*PhysicalHashJoin); ok { + join.SetChild(1-join.InnerChildIdx, ch) + } else { + p.SetChildren(ch) + } + p = ch + } + case *PhysicalHashJoin: + stack = append(stack, x.children[1-x.InnerChildIdx]) + err := untwistPlanAndRemoveUnionAll(stack, forest) + stack = stack[:len(stack)-1] + return errors.Trace(err) + case *PhysicalUnionAll: + for _, ch := range x.children { + stack = append(stack, ch) + err := untwistPlanAndRemoveUnionAll(stack, forest) + stack = stack[:len(stack)-1] + if err != nil { + return errors.Trace(err) + } + } + default: + if len(cur.Children()) != 1 { + return errors.Trace(errors.New("unexpected plan " + cur.ExplainID().String())) + } + ch := cur.Children()[0] + stack = append(stack, ch) + err := untwistPlanAndRemoveUnionAll(stack, forest) + stack = stack[:len(stack)-1] + return errors.Trace(err) + } + return nil } -func (e *mppTaskGenerator) generateMPPTasksForFragment(s *PhysicalExchangeSender) (tasks []*kv.MPPTask, err error) { - f, err := newFragment(s) +func buildFragments(s *PhysicalExchangeSender) ([]*Fragment, error) { + forest := make([]*PhysicalExchangeSender, 0, 1) + err := untwistPlanAndRemoveUnionAll([]PhysicalPlan{s}, &forest) if err != nil { return nil, errors.Trace(err) } + fragments := make([]*Fragment, 0, len(forest)) + for _, s := range forest { + f := &Fragment{ExchangeSender: s} + err = f.init(s) + if err != nil { + return nil, errors.Trace(err) + } + fragments = append(fragments, f) + } + return fragments, nil +} + +func (e *mppTaskGenerator) generateMPPTasksForExchangeSender(s *PhysicalExchangeSender) ([]*kv.MPPTask, []*Fragment, error) { + if cached, ok := e.cache[s.ID()]; ok { + return cached.tasks, cached.frags, nil + } + frags, err := buildFragments(s) + if err != nil { + return nil, nil, errors.Trace(err) + } + results := make([]*kv.MPPTask, 0, len(frags)) + for _, f := range frags { + tasks, err := e.generateMPPTasksForFragment(f) + if err != nil { + return nil, nil, errors.Trace(err) + } + results = append(results, tasks...) + } + e.frags = append(e.frags, frags...) + e.cache[s.ID()] = tasksAndFrags{results, frags} + return results, frags, nil +} + +func (e *mppTaskGenerator) generateMPPTasksForFragment(f *Fragment) (tasks []*kv.MPPTask, err error) { for _, r := range f.ExchangeReceivers { - r.Tasks, err = e.generateMPPTasksForFragment(r.GetExchangeSender()) + r.Tasks, r.frags, err = e.generateMPPTasksForExchangeSender(r.GetExchangeSender()) if err != nil { return nil, errors.Trace(err) } @@ -149,8 +255,9 @@ func (e *mppTaskGenerator) generateMPPTasksForFragment(s *PhysicalExchangeSender return nil, errors.New("cannot find mpp task") } for _, r := range f.ExchangeReceivers { - s := r.GetExchangeSender() - s.TargetTasks = tasks + for _, frag := range r.frags { + frag.ExchangeSender.TargetTasks = append(frag.ExchangeSender.TargetTasks, tasks...) + } } f.ExchangeSender.Tasks = tasks return tasks, nil diff --git a/planner/core/initialize.go b/planner/core/initialize.go index c63d4efa7ba31..7fff4f8b722b1 100644 --- a/planner/core/initialize.go +++ b/planner/core/initialize.go @@ -419,7 +419,7 @@ func (p PhysicalTableReader) Init(ctx sessionctx.Context, offset int) *PhysicalT if p.tablePlan != nil { p.TablePlans = flattenPushDownPlan(p.tablePlan) p.schema = p.tablePlan.Schema() - if p.StoreType == kv.TiFlash && !p.GetTableScan().KeepOrder { + if p.StoreType == kv.TiFlash && p.GetTableScan() != nil && !p.GetTableScan().KeepOrder { // When allow batch cop is 1, only agg / topN uses batch cop. // When allow batch cop is 2, every query uses batch cop. switch ctx.GetSessionVars().AllowBatchCop { diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index f47f43e79a35d..96a22dd092d46 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -1348,6 +1348,10 @@ func (b *PlanBuilder) buildProjection4Union(ctx context.Context, u *LogicalUnion b.optFlag |= flagEliminateProjection proj := LogicalProjection{Exprs: exprs, AvoidColumnEvaluator: true}.Init(b.ctx, b.getSelectOffset()) proj.SetSchema(u.schema.Clone()) + // reset the schema type to make the "not null" flag right. + for i, expr := range exprs { + proj.schema.Columns[i].RetType = expr.GetType() + } proj.SetChildren(child) u.children[childID] = proj } diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index a783d4f3d51db..292d9f8606d83 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -108,7 +108,10 @@ func (p *PhysicalTableReader) GetTableScan() *PhysicalTableScan { } else if chCnt == 1 { curPlan = curPlan.Children()[0] } else { - join := curPlan.(*PhysicalHashJoin) + join, ok := curPlan.(*PhysicalHashJoin) + if !ok { + return nil + } curPlan = join.children[1-join.globalChildIndex] } } @@ -883,6 +886,18 @@ type PhysicalExchangeReceiver struct { basePhysicalPlan Tasks []*kv.MPPTask + frags []*Fragment +} + +// Clone implment PhysicalPlan interface. +func (p *PhysicalExchangeReceiver) Clone() (PhysicalPlan, error) { + np := new(PhysicalExchangeReceiver) + base, err := p.basePhysicalPlan.cloneWithSelf(np) + if err != nil { + return nil, errors.Trace(err) + } + np.basePhysicalPlan = *base + return np, nil } // GetExchangeSender return the connected sender of this receiver. We assume that its child must be a receiver. @@ -897,10 +912,21 @@ type PhysicalExchangeSender struct { TargetTasks []*kv.MPPTask ExchangeType tipb.ExchangeType HashCols []*expression.Column - // Tasks is the mpp task for current PhysicalExchangeSender + // Tasks is the mpp task for current PhysicalExchangeSender. Tasks []*kv.MPPTask +} - Fragment *Fragment +// Clone implment PhysicalPlan interface. +func (p *PhysicalExchangeSender) Clone() (PhysicalPlan, error) { + np := new(PhysicalExchangeSender) + base, err := p.basePhysicalPlan.cloneWithSelf(np) + if err != nil { + return nil, errors.Trace(err) + } + np.basePhysicalPlan = *base + np.ExchangeType = p.ExchangeType + np.HashCols = p.HashCols + return np, nil } // Clone implements PhysicalPlan interface. @@ -951,6 +977,19 @@ func (p *PhysicalLimit) Clone() (PhysicalPlan, error) { // PhysicalUnionAll is the physical operator of UnionAll. type PhysicalUnionAll struct { physicalSchemaProducer + + mpp bool +} + +// Clone implements PhysicalPlan interface. +func (p *PhysicalUnionAll) Clone() (PhysicalPlan, error) { + cloned := new(PhysicalUnionAll) + base, err := p.physicalSchemaProducer.cloneWithSelf(cloned) + if err != nil { + return nil, err + } + cloned.physicalSchemaProducer = *base + return cloned, nil } // AggMppRunMode defines the running mode of aggregation in MPP diff --git a/planner/core/plan.go b/planner/core/plan.go index 2d38bfc375b65..f1449491c3cab 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -391,6 +391,9 @@ func (p *basePhysicalPlan) cloneWithSelf(newSelf PhysicalPlan) (*basePhysicalPla base.children = append(base.children, cloned) } for _, prop := range p.childrenReqProps { + if prop == nil { + continue + } base.childrenReqProps = append(base.childrenReqProps, prop.CloneEssentialFields()) } return base, nil diff --git a/planner/core/rule_inject_extra_projection.go b/planner/core/rule_inject_extra_projection.go index 2896a1dade0ff..911c531ceb4f0 100644 --- a/planner/core/rule_inject_extra_projection.go +++ b/planner/core/rule_inject_extra_projection.go @@ -14,6 +14,7 @@ package core import ( + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/kv" @@ -62,10 +63,43 @@ func (pe *projInjector) inject(plan PhysicalPlan) PhysicalPlan { plan = InjectProjBelowSort(p, p.ByItems) case *NominalSort: plan = TurnNominalSortIntoProj(p, p.OnlyColumn, p.ByItems) + case *PhysicalUnionAll: + plan = injectProjBelowUnion(p) } return plan } +func injectProjBelowUnion(un *PhysicalUnionAll) *PhysicalUnionAll { + if !un.mpp { + return un + } + for i, ch := range un.children { + exprs := make([]expression.Expression, len(ch.Schema().Columns)) + needChange := false + for i, dstCol := range un.schema.Columns { + dstType := dstCol.RetType + srcCol := ch.Schema().Columns[i] + srcCol.Index = i + srcType := srcCol.RetType + if !srcType.Equal(dstType) || !(mysql.HasNotNullFlag(dstType.Flag) == mysql.HasNotNullFlag(srcType.Flag)) { + exprs[i] = expression.BuildCastFunction4Union(un.ctx, srcCol, dstType) + needChange = true + } else { + exprs[i] = srcCol + } + } + if needChange { + proj := PhysicalProjection{ + Exprs: exprs, + }.Init(un.ctx, ch.statsInfo(), 0) + proj.SetSchema(un.schema.Clone()) + proj.SetChildren(ch) + un.children[i] = proj + } + } + return un +} + // wrapCastForAggFunc wraps the args of an aggregate function with a cast function. // If the mode is FinalMode or Partial2Mode, we do not need to wrap cast upon the args, // since the types of the args are already the expected. diff --git a/planner/core/task.go b/planner/core/task.go index 0138a0943231b..ae2ede66066ef 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -1265,7 +1265,30 @@ func (p *PhysicalProjection) attach2Task(tasks ...task) task { return t } +func (p *PhysicalUnionAll) attach2MppTasks(tasks ...task) task { + t := &mppTask{p: p} + childPlans := make([]PhysicalPlan, 0, len(tasks)) + var childMaxCost float64 + for _, tk := range tasks { + if mpp, ok := tk.(*mppTask); ok && !tk.invalid() { + childCost := mpp.cost() + if childCost > childMaxCost { + childMaxCost = childCost + } + childPlans = append(childPlans, mpp.plan()) + } else { + return invalidTask + } + } + p.SetChildren(childPlans...) + t.cst = childMaxCost + return t +} + func (p *PhysicalUnionAll) attach2Task(tasks ...task) task { + if _, ok := tasks[0].(*mppTask); ok { + return p.attach2MppTasks(tasks...) + } t := &rootTask{p: p} childPlans := make([]PhysicalPlan, 0, len(tasks)) var childMaxCost float64 diff --git a/store/copr/mpp.go b/store/copr/mpp.go index d1f125aed866d..6829a6e6a844f 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -240,6 +240,7 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *tikv.Backoffer, realResp := rpcResp.Resp.(*mpp.DispatchTaskResponse) if realResp.Error != nil { + logutil.BgLogger().Error("mpp dispatch response meet error", zap.String("error", realResp.Error.Msg)) m.sendError(errors.New(realResp.Error.Msg)) return } diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index 322b639f3c6e5..9a7d424e7956a 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -868,7 +868,7 @@ func getPlanIDFromExecutionSummary(summary *tipb.ExecutorExecutionSummary) (int, } // RecordOneCopTask records a specific cop tasks's execution detail. -func (e *RuntimeStatsColl) RecordOneCopTask(planID int, storeType string, address string, summary *tipb.ExecutorExecutionSummary) { +func (e *RuntimeStatsColl) RecordOneCopTask(planID int, storeType string, address string, summary *tipb.ExecutorExecutionSummary) int { // for TiFlash cop response, ExecutorExecutionSummary contains executor id, so if there is a valid executor id in // summary, use it overwrite the planID if id, valid := getPlanIDFromExecutionSummary(summary); valid { @@ -876,6 +876,7 @@ func (e *RuntimeStatsColl) RecordOneCopTask(planID int, storeType string, addres } copStats := e.GetOrCreateCopStats(planID, storeType) copStats.RecordOneCopTask(address, summary) + return planID } // RecordScanDetail records a specific cop tasks's cop detail. From 6c2dc4e8b5fa08c86a4fc6e1963924eed444fdb1 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Mon, 21 Jun 2021 17:50:53 +0800 Subject: [PATCH 20/27] planner/core: push down topn to mpp (#24081) (#25162) --- planner/core/exhaust_physical_plans.go | 3 + planner/core/integration_test.go | 9 ++- planner/core/task.go | 9 ++- .../testdata/integration_serial_suite_in.json | 7 ++- .../integration_serial_suite_out.json | 52 +++++++++++++++- .../core/testdata/integration_suite_out.json | 60 +++++++++---------- 6 files changed, 101 insertions(+), 39 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index c420ac77d4afa..ea73c8478b59e 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1994,6 +1994,9 @@ func (lt *LogicalTopN) getPhysTopN(prop *property.PhysicalProperty) []PhysicalPl if !lt.limitHints.preferLimitToCop { allTaskTypes = append(allTaskTypes, property.RootTaskType) } + if lt.ctx.GetSessionVars().AllowMPPExecution { + allTaskTypes = append(allTaskTypes, property.MppTaskType) + } ret := make([]PhysicalPlan, 0, len(allTaskTypes)) for _, tp := range allTaskTypes { resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: math.MaxFloat64} diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 8e21dfda05f26..d661a8e66fbad 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -356,6 +356,8 @@ func (s *testIntegrationSerialSuite) TestSelPushDownTiFlash(c *C) { } tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.tidb_allow_mpp = 0") + var input []string var output []struct { SQL string @@ -393,6 +395,7 @@ func (s *testIntegrationSerialSuite) TestPushDownToTiFlashWithKeepOrder(c *C) { } tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.tidb_allow_mpp = 0") var input []string var output []struct { SQL string @@ -1991,10 +1994,10 @@ func (s *testIntegrationSuite) TestAccessPathOnClusterIndex(c *C) { for i, tt := range input { s.testData.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain " + tt).Rows()) + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format='brief' " + tt).Rows()) output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) }) - tk.MustQuery("explain " + tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery("explain format='brief' " + tt).Check(testkit.Rows(output[i].Plan...)) tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Res...)) } } @@ -3070,7 +3073,7 @@ func (s *testIntegrationSerialSuite) TestMppJoinDecimal(c *C) { } } -func (s *testIntegrationSerialSuite) TestMppAggWithJoin(c *C) { +func (s *testIntegrationSerialSuite) TestMppAggTopNWithJoin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") diff --git a/planner/core/task.go b/planner/core/task.go index ae2ede66066ef..3bcbe6aa049be 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -1135,12 +1135,12 @@ func (p *PhysicalTopN) GetCost(count float64, isRoot bool) float64 { } // canPushDown checks if this topN can be pushed down. If each of the expression can be converted to pb, it can be pushed. -func (p *PhysicalTopN) canPushDown(cop *copTask) bool { +func (p *PhysicalTopN) canPushDown(storeTp kv.StoreType) bool { exprs := make([]expression.Expression, 0, len(p.ByItems)) for _, item := range p.ByItems { exprs = append(exprs, item.Expr) } - return expression.CanExprsPushDown(p.ctx.GetSessionVars().StmtCtx, exprs, p.ctx.GetClient(), cop.getStoreType()) + return expression.CanExprsPushDown(p.ctx.GetSessionVars().StmtCtx, exprs, p.ctx.GetClient(), storeTp) } func (p *PhysicalTopN) allColsFromSchema(schema *expression.Schema) bool { @@ -1210,7 +1210,7 @@ func (p *PhysicalTopN) getPushedDownTopN(childPlan PhysicalPlan) *PhysicalTopN { func (p *PhysicalTopN) attach2Task(tasks ...task) task { t := tasks[0].copy() inputCount := t.count() - if copTask, ok := t.(*copTask); ok && p.canPushDown(copTask) && len(copTask.rootTaskConds) == 0 { + if copTask, ok := t.(*copTask); ok && p.canPushDown(copTask.getStoreType()) && len(copTask.rootTaskConds) == 0 { // If all columns in topN are from index plan, we push it to index plan, otherwise we finish the index plan and // push it to table plan. var pushedDownTopN *PhysicalTopN @@ -1223,6 +1223,9 @@ func (p *PhysicalTopN) attach2Task(tasks ...task) task { copTask.tablePlan = pushedDownTopN } copTask.addCost(pushedDownTopN.GetCost(inputCount, false)) + } else if mppTask, ok := t.(*mppTask); ok && p.canPushDown(kv.TiFlash) { + pushedDownTopN := p.getPushedDownTopN(mppTask.p) + mppTask.p = pushedDownTopN } rootTask := t.convertToRootTask(p.ctx) rootTask.addCost(p.GetCost(rootTask.count(), true)) diff --git a/planner/core/testdata/integration_serial_suite_in.json b/planner/core/testdata/integration_serial_suite_in.json index 8733b9a97e8c2..c064aa3b58edc 100644 --- a/planner/core/testdata/integration_serial_suite_in.json +++ b/planner/core/testdata/integration_serial_suite_in.json @@ -249,7 +249,7 @@ ] }, { - "name": "TestMppAggWithJoin", + "name": "TestMppAggTopNWithJoin", "cases": [ "desc format = 'brief' select * from t join ( select count(*), id from t group by id) as A on A.id = t.id", "desc format = 'brief' select * from t join ( select count(*)+id as v from t group by id) as A on A.v = t.id", @@ -257,7 +257,10 @@ "desc format = 'brief' select * from t join ( select /*+ hash_agg()*/ count(*) as a from t) as A on A.a = t.id", "desc format = 'brief' select sum(b) from (select t.id, t1.id as b from t join t t1 on t.id=t1.id)A group by id", "desc format = 'brief' select * from (select id from t group by id) C join (select sum(value),id from t group by id)B on C.id=B.id", - "desc format = 'brief' select * from (select id from t group by id) C join (select sum(b),id from (select t.id, t1.id as b from t join (select id, count(*) as c from t group by id) t1 on t.id=t1.id)A group by id)B on C.id=b.id" + "desc format = 'brief' select * from (select id from t group by id) C join (select sum(b),id from (select t.id, t1.id as b from t join (select id, count(*) as c from t group by id) t1 on t.id=t1.id)A group by id)B on C.id=b.id", + "desc format = 'brief' select * from t join t t1 on t.id = t1.id order by t.value limit 1", + "desc format = 'brief' select * from t join t t1 on t.id = t1.id order by t.value % 100 limit 1", + "desc format = 'brief' select count(*) from (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 20) v group by v.v1" ] }, { diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index 20ab278ed0cb2..df1a3c0a729b1 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -2097,7 +2097,7 @@ ] }, { - "Name": "TestMppAggWithJoin", + "Name": "TestMppAggTopNWithJoin", "Cases": [ { "SQL": "desc format = 'brief' select * from t join ( select count(*), id from t group by id) as A on A.id = t.id", @@ -2251,6 +2251,56 @@ " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] + }, + { + "SQL": "desc format = 'brief' select * from t join t t1 on t.id = t1.id order by t.value limit 1", + "Plan": [ + "TopN 1.00 root test.t.value, offset:0, count:1", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─TopN 1.00 batchCop[tiflash] test.t.value, offset:0, count:1", + " └─HashJoin 12487.50 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", + " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t join t t1 on t.id = t1.id order by t.value % 100 limit 1", + "Plan": [ + "Projection 1.00 root test.t.id, test.t.value, test.t.id, test.t.value", + "└─TopN 1.00 root Column#7, offset:0, count:1", + " └─Projection 12487.50 root test.t.id, test.t.value, test.t.id, test.t.value, mod(test.t.value, 100)->Column#7", + " └─TableReader 12487.50 root data:ExchangeSender", + " └─ExchangeSender 12487.50 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", + " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select count(*) from (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 20) v group by v.v1", + "Plan": [ + "HashAgg 20.00 root group by:test.t.value, funcs:count(1)->Column#7", + "└─TopN 20.00 root test.t.value, offset:0, count:20", + " └─TableReader 20.00 root data:ExchangeSender", + " └─ExchangeSender 20.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─TopN 20.00 batchCop[tiflash] test.t.value, offset:0, count:20", + " └─HashJoin 12487.50 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", + " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo" + ] } ] }, diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index 652832586f3db..c73c994340ea0 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -901,8 +901,8 @@ { "SQL": "select * from t1", "Plan": [ - "TableReader_5 3.00 root data:TableFullScan_4", - "└─TableFullScan_4 3.00 cop[tikv] table:t1 keep order:false" + "TableReader 3.00 root data:TableFullScan", + "└─TableFullScan 3.00 cop[tikv] table:t1 keep order:false" ], "Res": [ "1 111 1.1000000000 11", @@ -913,8 +913,8 @@ { "SQL": "select * from t1 where t1.a >= 1 and t1.a < 4", "Plan": [ - "TableReader_6 3.00 root data:TableRangeScan_5", - "└─TableRangeScan_5 3.00 cop[tikv] table:t1 range:[1,4), keep order:false" + "TableReader 3.00 root data:TableRangeScan", + "└─TableRangeScan 3.00 cop[tikv] table:t1 range:[1,4), keep order:false" ], "Res": [ "1 111 1.1000000000 11", @@ -925,8 +925,8 @@ { "SQL": "select * from t1 where t1.a = 1 and t1.b < \"333\"", "Plan": [ - "TableReader_6 0.67 root data:TableRangeScan_5", - "└─TableRangeScan_5 0.67 cop[tikv] table:t1 range:[1 -inf,1 \"333\"), keep order:false" + "TableReader 0.67 root data:TableRangeScan", + "└─TableRangeScan 0.67 cop[tikv] table:t1 range:[1 -inf,1 \"333\"), keep order:false" ], "Res": [ "1 111 1.1000000000 11" @@ -935,8 +935,8 @@ { "SQL": "select t1.a, t1.b, t1.c from t1 where t1.c = 3.3", "Plan": [ - "IndexReader_6 1.00 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 1.00 cop[tikv] table:t1, index:c(c) range:[3.3000000000,3.3000000000], keep order:false" + "IndexReader 1.00 root index:IndexRangeScan", + "└─IndexRangeScan 1.00 cop[tikv] table:t1, index:c(c) range:[3.3000000000,3.3000000000], keep order:false" ], "Res": [ "3 333 3.3000000000" @@ -945,8 +945,8 @@ { "SQL": "select t1.b, t1.c from t1 where t1.c = 2.2", "Plan": [ - "IndexReader_6 1.00 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 1.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], keep order:false" + "IndexReader 1.00 root index:IndexRangeScan", + "└─IndexRangeScan 1.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], keep order:false" ], "Res": [ "222 2.2000000000" @@ -955,9 +955,9 @@ { "SQL": "select /*+ use_index(t1, c) */ * from t1", "Plan": [ - "IndexLookUp_6 3.00 root ", - "├─IndexFullScan_4(Build) 3.00 cop[tikv] table:t1, index:c(c) keep order:false", - "└─TableRowIDScan_5(Probe) 3.00 cop[tikv] table:t1 keep order:false" + "IndexLookUp 3.00 root ", + "├─IndexFullScan(Build) 3.00 cop[tikv] table:t1, index:c(c) keep order:false", + "└─TableRowIDScan(Probe) 3.00 cop[tikv] table:t1 keep order:false" ], "Res": [ "1 111 1.1000000000 11", @@ -968,9 +968,9 @@ { "SQL": "select * from t1 use index(c) where t1.c in (2.2, 3.3)", "Plan": [ - "IndexLookUp_7 2.00 root ", - "├─IndexRangeScan_5(Build) 2.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], [3.3000000000,3.3000000000], keep order:false", - "└─TableRowIDScan_6(Probe) 2.00 cop[tikv] table:t1 keep order:false" + "IndexLookUp 2.00 root ", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], [3.3000000000,3.3000000000], keep order:false", + "└─TableRowIDScan(Probe) 2.00 cop[tikv] table:t1 keep order:false" ], "Res": [ "2 222 2.2000000000 12", @@ -980,8 +980,8 @@ { "SQL": "select * from t1 where t1.a = 1 order by b", "Plan": [ - "TableReader_12 1.00 root data:TableRangeScan_11", - "└─TableRangeScan_11 1.00 cop[tikv] table:t1 range:[1,1], keep order:true" + "TableReader 1.00 root data:TableRangeScan", + "└─TableRangeScan 1.00 cop[tikv] table:t1 range:[1,1], keep order:true" ], "Res": [ "1 111 1.1000000000 11" @@ -990,10 +990,10 @@ { "SQL": "select * from t1 order by a, b limit 1", "Plan": [ - "Limit_10 1.00 root offset:0, count:1", - "└─TableReader_20 1.00 root data:Limit_19", - " └─Limit_19 1.00 cop[tikv] offset:0, count:1", - " └─TableFullScan_18 1.00 cop[tikv] table:t1 keep order:true" + "Limit 1.00 root offset:0, count:1", + "└─TableReader 1.00 root data:Limit", + " └─Limit 1.00 cop[tikv] offset:0, count:1", + " └─TableFullScan 1.00 cop[tikv] table:t1 keep order:true" ], "Res": [ "1 111 1.1000000000 11" @@ -1002,10 +1002,10 @@ { "SQL": "select /*+ use_index_merge(t1 primary, c) */ * from t1 where t1.a >= 1 or t1.c = 2.2", "Plan": [ - "IndexMerge_8 3.00 root ", - "├─TableRangeScan_5(Build) 3.00 cop[tikv] table:t1 range:[1,+inf], keep order:false", - "├─IndexRangeScan_6(Build) 1.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], keep order:false", - "└─TableRowIDScan_7(Probe) 3.00 cop[tikv] table:t1 keep order:false" + "IndexMerge 3.00 root ", + "├─TableRangeScan(Build) 3.00 cop[tikv] table:t1 range:[1,+inf], keep order:false", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, index:c(c) range:[2.2000000000,2.2000000000], keep order:false", + "└─TableRowIDScan(Probe) 3.00 cop[tikv] table:t1 keep order:false" ], "Res": [ "1 111 1.1000000000 11", @@ -1016,10 +1016,10 @@ { "SQL": "select /*+ use_index_merge(t1 primary, c) */ * from t1 where t1.a = 1 and t1.b = '111' or t1.c = 3.3", "Plan": [ - "IndexMerge_8 2.11 root ", - "├─TableRangeScan_5(Build) 1.00 cop[tikv] table:t1 range:[1 \"111\",1 \"111\"], keep order:false", - "├─IndexRangeScan_6(Build) 1.00 cop[tikv] table:t1, index:c(c) range:[3.3000000000,3.3000000000], keep order:false", - "└─TableRowIDScan_7(Probe) 2.11 cop[tikv] table:t1 keep order:false" + "IndexMerge 2.11 root ", + "├─TableRangeScan(Build) 1.00 cop[tikv] table:t1 range:[1 \"111\",1 \"111\"], keep order:false", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, index:c(c) range:[3.3000000000,3.3000000000], keep order:false", + "└─TableRowIDScan(Probe) 2.11 cop[tikv] table:t1 keep order:false" ], "Res": [ "1 111 1.1000000000 11", From 32cd4a90dc5dcef6c8cd2a6b4c549b6f01e1337c Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Mon, 21 Jun 2021 18:04:53 +0800 Subject: [PATCH 21/27] planner/core: support limit push down (#24757) (#25159) --- planner/core/exhaust_physical_plans.go | 6 +++++- planner/core/task.go | 16 ++++++++++++++-- 2 files changed, 19 insertions(+), 3 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index ea73c8478b59e..4dc7f55cf63e8 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1843,7 +1843,8 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC EqualConditions: p.EqualConditions, storeTp: kv.TiFlash, mppShuffleJoin: !useBCJ, - }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, childrenProps...) + // Mpp Join has quite heavy cost. Even limit might not suspend it in time, so we dont scale the count. + }.Init(p.ctx, p.stats, p.blockOffset, childrenProps...) return []PhysicalPlan{join} } @@ -2494,6 +2495,9 @@ func (p *LogicalLimit) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([] if !p.limitHints.preferLimitToCop { allTaskTypes = append(allTaskTypes, property.RootTaskType) } + if p.canPushToCop(kv.TiFlash) && p.ctx.GetSessionVars().AllowMPPExecution { + allTaskTypes = append(allTaskTypes, property.MppTaskType) + } ret := make([]PhysicalPlan, 0, len(allTaskTypes)) for _, tp := range allTaskTypes { resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: float64(p.Count + p.Offset)} diff --git a/planner/core/task.go b/planner/core/task.go index 3bcbe6aa049be..835c4c21c70b8 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -1074,6 +1074,14 @@ func (p *PhysicalLimit) attach2Task(tasks ...task) task { } t = cop.convertToRootTask(p.ctx) sunk = p.sinkIntoIndexLookUp(t) + } else if mpp, ok := t.(*mppTask); ok { + newCount := p.Offset + p.Count + childProfile := mpp.plan().statsInfo() + stats := deriveLimitStats(childProfile, float64(newCount)) + pushedDownLimit := PhysicalLimit{Count: newCount}.Init(p.ctx, stats, p.blockOffset) + mpp = attachPlan2Task(pushedDownLimit, mpp).(*mppTask) + pushedDownLimit.SetSchema(pushedDownLimit.children[0].Schema()) + t = mpp.convertToRootTask(p.ctx) } if sunk { return t @@ -1999,10 +2007,14 @@ func (t *mppTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { StoreType: kv.TiFlash, }.Init(ctx, t.p.SelectBlockOffset()) p.stats = t.p.statsInfo() - return &rootTask{ + + cst := t.cst + t.count()*ctx.GetSessionVars().NetworkFactor + cst = cst / p.ctx.GetSessionVars().CopTiFlashConcurrencyFactor + rt := &rootTask{ p: p, - cst: t.cst / 20, // TODO: This is tricky because mpp doesn't run in a coprocessor way. + cst: cst, } + return rt } func (t *mppTask) needEnforce(prop *property.PhysicalProperty) bool { From c8a80a3e51793cf42067664e2d18f3c2ed610de8 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Tue, 22 Jun 2021 00:02:53 +0800 Subject: [PATCH 22/27] planner: make sure limit outputs no more columns than its child (#25345) (#25517) --- planner/core/integration_test.go | 9 +++++++++ planner/core/rule_column_pruning.go | 10 ++++++++-- 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index d661a8e66fbad..4fb7ad3c45587 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3229,6 +3229,15 @@ func (s *testIntegrationSuite) TestIssue24281(c *C) { "UNION select 1 as v1, 2 as v2") } +func (s *testIntegrationSuite) TestLimitWindowColPrune(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int)") + tk.MustExec("insert into t values(1)") + tk.MustQuery("select count(a) f1, row_number() over (order by count(a)) as f2 from t limit 1").Check(testkit.Rows("1 1")) +} + func (s *testIntegrationSerialSuite) TestMergeContinuousSelections(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index 69f2c9ec0bbb6..b10516bd6c75a 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -428,8 +428,14 @@ func (p *LogicalLimit) PruneColumns(parentUsedCols []*expression.Column) error { return nil } - p.inlineProjection(parentUsedCols) - return p.children[0].PruneColumns(parentUsedCols) + savedUsedCols := make([]*expression.Column, len(parentUsedCols)) + copy(savedUsedCols, parentUsedCols) + if err := p.children[0].PruneColumns(parentUsedCols); err != nil { + return err + } + p.schema = nil + p.inlineProjection(savedUsedCols) + return nil } func (*columnPruner) name() string { From ef3e4fd8917250d8cfadbf90963ff7403bff3e50 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Tue, 22 Jun 2021 11:22:54 +0800 Subject: [PATCH 23/27] executor: make the ParallelApply be safe to be called again after returning empty results (#24935) (#25011) --- executor/parallel_apply.go | 7 +++++++ executor/parallel_apply_test.go | 11 +++++++++++ 2 files changed, 18 insertions(+) diff --git a/executor/parallel_apply.go b/executor/parallel_apply.go index 39c7ea0da3a9a..9819b81c8100b 100644 --- a/executor/parallel_apply.go +++ b/executor/parallel_apply.go @@ -70,6 +70,7 @@ type ParallelNestedLoopApplyExec struct { // fields about concurrency control concurrency int started uint32 + drained uint32 // drained == true indicates there is no more data freeChkCh chan *chunk.Chunk resultChkCh chan result outerRowCh chan outerRow @@ -132,6 +133,11 @@ func (e *ParallelNestedLoopApplyExec) Open(ctx context.Context) error { // Next implements the Executor interface. func (e *ParallelNestedLoopApplyExec) Next(ctx context.Context, req *chunk.Chunk) (err error) { + if atomic.LoadUint32(&e.drained) == 1 { + req.Reset() + return nil + } + if atomic.CompareAndSwapUint32(&e.started, 0, 1) { e.workerWg.Add(1) go e.outerWorker(ctx) @@ -149,6 +155,7 @@ func (e *ParallelNestedLoopApplyExec) Next(ctx context.Context, req *chunk.Chunk } if result.chk == nil { // no more data req.Reset() + atomic.StoreUint32(&e.drained, 1) return nil } req.SwapColumns(result.chk) diff --git a/executor/parallel_apply_test.go b/executor/parallel_apply_test.go index b849d3d961043..e8002d52fee36 100644 --- a/executor/parallel_apply_test.go +++ b/executor/parallel_apply_test.go @@ -597,3 +597,14 @@ func (s *testSuite) TestApplyGoroutinePanic(c *C) { c.Assert(failpoint.Disable(panicPath), IsNil) } } + +func (s *testSuite) TestIssue24930(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("set tidb_enable_parallel_apply=true") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1(a int)") + tk.MustExec("create table t2(a int)") + tk.MustQuery(`select case when t1.a is null + then (select t2.a from t2 where t2.a = t1.a limit 1) else t1.a end a + from t1 where t1.a=1 order by a limit 1`).Check(testkit.Rows()) // can return an empty result instead of hanging forever +} From 76eae7475cb73bc85f533ea464cd4f1b8380dc32 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Tue, 22 Jun 2021 11:44:54 +0800 Subject: [PATCH 24/27] store: avoid sessions of GC being affected by global variables (#24975) (#25609) --- store/gcworker/gc_worker.go | 1 + 1 file changed, 1 insertion(+) diff --git a/store/gcworker/gc_worker.go b/store/gcworker/gc_worker.go index a1f202ceb6863..e49d06708c613 100644 --- a/store/gcworker/gc_worker.go +++ b/store/gcworker/gc_worker.go @@ -227,6 +227,7 @@ func createSession(store kv.Storage) session.Session { } // Disable privilege check for gc worker session. privilege.BindPrivilegeManager(se, nil) + se.GetSessionVars().CommonGlobalLoaded = true se.GetSessionVars().InRestrictedSQL = true return se } From 7ac0ec24ceae7c730fc18c24632d3669c518c094 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Tue, 22 Jun 2021 11:56:53 +0800 Subject: [PATCH 25/27] planner: generate correct number of rows when all agg funcs are pruned (#24937) (#25062) --- cmd/explaintest/r/explain_easy.result | 21 +++--- planner/core/integration_test.go | 23 +++++++ planner/core/rule_column_pruning.go | 21 ++++-- .../core/testdata/integration_suite_in.json | 15 +++++ .../core/testdata/integration_suite_out.json | 65 +++++++++++++++++++ 5 files changed, 131 insertions(+), 14 deletions(-) diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index 313e41a006658..94e8da6eaa011 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -194,31 +194,32 @@ test t4 1 expr_idx 1 NULL NULL (`a` + `b` + 1) 2 YES NO explain format = 'brief' select count(1) from (select count(1) from (select * from t1 where c3 = 100) k) k2; id estRows task access object operator info StreamAgg 1.00 root funcs:count(1)->Column#5 -└─StreamAgg 1.00 root funcs:firstrow(Column#9)->Column#7 +└─StreamAgg 1.00 root funcs:count(Column#9)->Column#7 └─TableReader 1.00 root data:StreamAgg - └─StreamAgg 1.00 cop[tikv] funcs:firstrow(1)->Column#9 + └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#9 └─Selection 10.00 cop[tikv] eq(test.t1.c3, 100) └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo explain format = 'brief' select 1 from (select count(c2), count(c3) from t1) k; id estRows task access object operator info Projection 1.00 root 1->Column#6 -└─StreamAgg 1.00 root funcs:firstrow(Column#14)->Column#9 +└─StreamAgg 1.00 root funcs:count(Column#14)->Column#9 └─TableReader 1.00 root data:StreamAgg - └─StreamAgg 1.00 cop[tikv] funcs:firstrow(1)->Column#14 + └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#14 └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo explain format = 'brief' select count(1) from (select max(c2), count(c3) as m from t1) k; id estRows task access object operator info StreamAgg 1.00 root funcs:count(1)->Column#6 -└─StreamAgg 1.00 root funcs:firstrow(Column#13)->Column#8 +└─StreamAgg 1.00 root funcs:count(Column#13)->Column#8 └─TableReader 1.00 root data:StreamAgg - └─StreamAgg 1.00 cop[tikv] funcs:firstrow(1)->Column#13 + └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#13 └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo explain format = 'brief' select count(1) from (select count(c2) from t1 group by c3) k; id estRows task access object operator info StreamAgg 1.00 root funcs:count(1)->Column#5 -└─HashAgg 8000.00 root group by:test.t1.c3, funcs:firstrow(1)->Column#7 - └─TableReader 10000.00 root data:TableFullScan - └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─HashAgg 8000.00 root group by:test.t1.c3, funcs:count(Column#9)->Column#7 + └─TableReader 8000.00 root data:HashAgg + └─HashAgg 8000.00 cop[tikv] group by:test.t1.c3, funcs:count(1)->Column#9 + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo set @@session.tidb_opt_insubq_to_join_and_agg=0; explain format = 'brief' select sum(t1.c1 in (select c1 from t2)) from t1; id estRows task access object operator info @@ -498,7 +499,7 @@ PRIMARY KEY (`id`) explain format = 'brief' SELECT COUNT(1) FROM (SELECT COALESCE(b.region_name, '不详') region_name, SUM(a.registration_num) registration_num FROM (SELECT stat_date, show_date, region_id, 0 registration_num FROM test01 WHERE period = 1 AND stat_date >= 20191202 AND stat_date <= 20191202 UNION ALL SELECT stat_date, show_date, region_id, registration_num registration_num FROM test01 WHERE period = 1 AND stat_date >= 20191202 AND stat_date <= 20191202) a LEFT JOIN test02 b ON a.region_id = b.id WHERE registration_num > 0 AND a.stat_date >= '20191202' AND a.stat_date <= '20191202' GROUP BY a.stat_date , a.show_date , COALESCE(b.region_name, '不详') ) JLS; id estRows task access object operator info StreamAgg 1.00 root funcs:count(1)->Column#22 -└─HashAgg 8000.00 root group by:Column#32, Column#33, Column#34, funcs:firstrow(1)->Column#31 +└─HashAgg 8000.00 root group by:Column#32, Column#33, Column#34, funcs:count(1)->Column#31 └─Projection 10000.01 root Column#14, Column#15, coalesce(test.test02.region_name, 不详)->Column#34 └─HashJoin 10000.01 root left outer join, equal:[eq(Column#16, test.test02.id)] ├─TableReader(Build) 10000.00 root data:TableFullScan diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 4fb7ad3c45587..fc44a6f5f2a4d 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -178,6 +178,29 @@ func (s *testIntegrationSuite) TestPushLimitDownIndexLookUpReader(c *C) { } } +func (s *testIntegrationSuite) TestAggColumnPrune(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int)") + tk.MustExec("insert into t values(1),(2)") + + var input []string + var output []struct { + SQL string + Res []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) + } +} + func (s *testIntegrationSuite) TestIsFromUnixtimeNullRejective(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index b10516bd6c75a..e118738a9de0c 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -88,7 +88,11 @@ func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column) child := la.children[0] used := expression.GetUsedList(parentUsedCols, la.Schema()) + allFirstRow := true for i := len(used) - 1; i >= 0; i-- { + if la.AggFuncs[i].Name != ast.AggFuncFirstRow { + allFirstRow = false + } if !used[i] { la.schema.Columns = append(la.schema.Columns[:i], la.schema.Columns[i+1:]...) la.AggFuncs = append(la.AggFuncs[:i], la.AggFuncs[i+1:]...) @@ -103,15 +107,24 @@ func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column) selfUsedCols = append(selfUsedCols, cols...) } if len(la.AggFuncs) == 0 { - // If all the aggregate functions are pruned, we should add an aggregate function to keep the correctness. - one, err := aggregation.NewAggFuncDesc(la.ctx, ast.AggFuncFirstRow, []expression.Expression{expression.NewOne()}, false) + // If all the aggregate functions are pruned, we should add an aggregate function to maintain the info of row numbers. + // For all the aggregate functions except `first_row`, if we have an empty table defined as t(a,b), + // `select agg(a) from t` would always return one row, while `select agg(a) from t group by b` would return empty. + // For `first_row` which is only used internally by tidb, `first_row(a)` would always return empty for empty input now. + var err error + var newAgg *aggregation.AggFuncDesc + if allFirstRow { + newAgg, err = aggregation.NewAggFuncDesc(la.ctx, ast.AggFuncFirstRow, []expression.Expression{expression.NewOne()}, false) + } else { + newAgg, err = aggregation.NewAggFuncDesc(la.ctx, ast.AggFuncCount, []expression.Expression{expression.NewOne()}, false) + } if err != nil { return err } - la.AggFuncs = []*aggregation.AggFuncDesc{one} + la.AggFuncs = []*aggregation.AggFuncDesc{newAgg} col := &expression.Column{ UniqueID: la.ctx.GetSessionVars().AllocPlanColumnID(), - RetType: one.RetTp, + RetType: newAgg.RetTp, } la.schema.Columns = []*expression.Column{col} } diff --git a/planner/core/testdata/integration_suite_in.json b/planner/core/testdata/integration_suite_in.json index 3dbebbfefbfe0..3ba7037a3bc1d 100644 --- a/planner/core/testdata/integration_suite_in.json +++ b/planner/core/testdata/integration_suite_in.json @@ -19,6 +19,21 @@ "explain format = 'brief' select * from t t1 left join t t2 on t1.a=t2.a where from_unixtime(t2.b);" ] }, + { + "name": "TestAggColumnPrune", + "cases": [ + "select count(1) from t join (select count(1) from t where false) as tmp", + "select count(1) from t join (select max(a) from t where false) as tmp", + "select count(1) from t join (select min(a) from t where false) as tmp", + "select count(1) from t join (select sum(a) from t where false) as tmp", + "select count(1) from t join (select avg(a) from t where false) as tmp", + "select count(1) from t join (select count(1) from t where false group by a) as tmp", + "select count(1) from t join (select max(a) from t where false group by a) as tmp", + "select count(1) from t join (select min(a) from t where false group by a) as tmp", + "select count(1) from t join (select sum(a) from t where false group by a) as tmp", + "select count(1) from t join (select avg(a) from t where false group by a) as tmp" + ] + }, { "name": "TestIndexJoinInnerIndexNDV", "cases": [ diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index c73c994340ea0..cc5f2f8872157 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -63,6 +63,71 @@ } ] }, + { + "Name": "TestAggColumnPrune", + "Cases": [ + { + "SQL": "select count(1) from t join (select count(1) from t where false) as tmp", + "Res": [ + "2" + ] + }, + { + "SQL": "select count(1) from t join (select max(a) from t where false) as tmp", + "Res": [ + "2" + ] + }, + { + "SQL": "select count(1) from t join (select min(a) from t where false) as tmp", + "Res": [ + "2" + ] + }, + { + "SQL": "select count(1) from t join (select sum(a) from t where false) as tmp", + "Res": [ + "2" + ] + }, + { + "SQL": "select count(1) from t join (select avg(a) from t where false) as tmp", + "Res": [ + "2" + ] + }, + { + "SQL": "select count(1) from t join (select count(1) from t where false group by a) as tmp", + "Res": [ + "0" + ] + }, + { + "SQL": "select count(1) from t join (select max(a) from t where false group by a) as tmp", + "Res": [ + "0" + ] + }, + { + "SQL": "select count(1) from t join (select min(a) from t where false group by a) as tmp", + "Res": [ + "0" + ] + }, + { + "SQL": "select count(1) from t join (select sum(a) from t where false group by a) as tmp", + "Res": [ + "0" + ] + }, + { + "SQL": "select count(1) from t join (select avg(a) from t where false group by a) as tmp", + "Res": [ + "0" + ] + } + ] + }, { "Name": "TestIndexJoinInnerIndexNDV", "Cases": [ From 660bacede31be869ba33cb6e4d5aec76430b384a Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Tue, 22 Jun 2021 13:10:54 +0800 Subject: [PATCH 26/27] planner: select distinct should bypass batchget (#25477) (#25533) --- executor/point_get_test.go | 14 ++++++++++++++ planner/core/point_get_plan.go | 2 +- 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/executor/point_get_test.go b/executor/point_get_test.go index 0360add8228e5..9895364c89763 100644 --- a/executor/point_get_test.go +++ b/executor/point_get_test.go @@ -158,6 +158,20 @@ func (s *testPointGetSuite) TestPointGetDataTooLong(c *C) { tk.MustExec("drop table if exists PK_1389;") } +// issue #25320 +func (s *testPointGetSuite) TestDistinctPlan(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists test_distinct;") + tk.MustExec(`CREATE TABLE test_distinct ( + id bigint(18) NOT NULL COMMENT '主键', + b bigint(18) NOT NULL COMMENT '用户ID', + PRIMARY KEY (id) + ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci;`) + tk.MustExec("insert into test_distinct values (123456789101112131,223456789101112131),(123456789101112132,223456789101112131);") + tk.MustQuery("select distinct b from test_distinct where id in (123456789101112131,123456789101112132);").Check(testkit.Rows("223456789101112131")) +} + func (s *testPointGetSuite) TestPointGetCharPK(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec(`use test;`) diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index 6f3d524ca7490..5e4e0e46f864b 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -665,7 +665,7 @@ func newBatchPointGetPlan( func tryWhereIn2BatchPointGet(ctx sessionctx.Context, selStmt *ast.SelectStmt) *BatchPointGetPlan { if selStmt.OrderBy != nil || selStmt.GroupBy != nil || - selStmt.Limit != nil || selStmt.Having != nil || + selStmt.Limit != nil || selStmt.Having != nil || selStmt.Distinct || len(selStmt.WindowSpecs) > 0 { return nil } From 9030fbbc86ba320664ec6309e169c62c2139a132 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Tue, 22 Jun 2021 14:06:53 +0800 Subject: [PATCH 27/27] planner/core: remove random test to reduce CI time (#24207) (#24241) --- planner/core/partition_pruner_test.go | 183 +++++++------------------- 1 file changed, 47 insertions(+), 136 deletions(-) diff --git a/planner/core/partition_pruner_test.go b/planner/core/partition_pruner_test.go index 0d9a66c7dbbda..9b7898e23b891 100644 --- a/planner/core/partition_pruner_test.go +++ b/planner/core/partition_pruner_test.go @@ -16,7 +16,6 @@ package core_test import ( "bytes" "fmt" - "math/rand" "sort" "strings" @@ -311,142 +310,54 @@ func (s *testPartitionPruneSuit) getFieldValue(prefix, row string) string { func (s *testPartitionPruneSuit) TestListColumnsPartitionPrunerRandom(c *C) { tk := testkit.NewTestKit(c, s.store) - for count := 0; count < 5; count++ { - partitionNum := rand.Intn(10) + 1 - valueNum := rand.Intn(10) + 1 - condNum := 20 - - partitionDefs := make([][]string, partitionNum) - for id := 0; id < valueNum; id++ { - for a := 0; a < valueNum; a++ { - for b := 0; b < valueNum; b++ { - idx := rand.Intn(partitionNum) - partitionDefs[idx] = append(partitionDefs[idx], fmt.Sprintf("(%v,%v,%v)", b, id, a)) - } - } - } - validIdx := 0 - for _, def := range partitionDefs { - if len(def) > 0 { - partitionDefs[validIdx] = def - validIdx++ - } - } - partitionDefs = partitionDefs[:validIdx] - createSQL := bytes.NewBuffer(make([]byte, 0, 1024*1024)) - // Generate table definition. - colNames := []string{"id", "a", "b"} - createSQL.WriteString("create table t1 (id int, a int, b int") - // Generate Index definition. - if rand.Int()%2 == 0 { - createSQL.WriteString(", index (") - n := rand.Intn(len(colNames)) + 1 - cols := map[string]struct{}{} - for i := 0; i < n; i++ { - col := colNames[rand.Intn(len(colNames))] - cols[col] = struct{}{} - } - cnt := 0 - for col := range cols { - if cnt > 0 { - createSQL.WriteString(",") - } - createSQL.WriteString(col) - cnt++ - } - createSQL.WriteString(")") - } - createSQL.WriteString(" ) partition by list columns (b, id, a) (") - - for i := range partitionDefs { - if i > 0 { - createSQL.WriteString(",") - } - createSQL.WriteString(fmt.Sprintf("partition p%v values in (", i)) - for idx, v := range partitionDefs[i] { - if idx > 0 { - createSQL.WriteString(",") - } - createSQL.WriteString(v) - } - createSQL.WriteString(")") - } - createSQL.WriteString(")") - - // Create table. - tk.MustExec("drop database if exists test_partition;") - tk.MustExec("create database test_partition") - tk.MustExec("use test_partition") - tk.MustExec("set @@session.tidb_enable_list_partition = ON") - tk.MustExec(createSQL.String()) - - tk1 := testkit.NewTestKit(c, s.store) - tk1.MustExec("drop database if exists test_partition_1;") - tk1.MustExec("create database test_partition_1") - tk1.MustExec("use test_partition_1") - tk1.MustExec("create table t1 (id int, a int, b int)") - - // prepare data. - for _, def := range partitionDefs { - insert := fmt.Sprintf("insert into t1 (b,id,a) values %v", strings.Join(def, ",")) - tk.MustExec(insert) - tk1.MustExec(insert) - - // Test query without condition - query := fmt.Sprintf("select * from t1 order by id,a,b") - tk.MustQuery(query).Check(tk1.MustQuery(query).Rows()) - } + valueNum := 10 + // Create table. + tk.MustExec("drop database if exists test_partition;") + tk.MustExec("create database test_partition") + tk.MustExec("use test_partition") + tk.MustExec("set @@session.tidb_enable_list_partition = ON") + tk.MustExec("create table t1 (id int, a int, b int ) partition by list columns (b, id, a) (partition p0 values in ((1,0,2),(2,0,2),(0,1,0),(1,1,0),(2,1,0),(0,1,1),(0,1,2),(0,2,0),(1,2,0)),partition p1 values in ((1,0,1),(0,0,2),(2,1,1),(2,1,2),(2,2,1),(1,2,2),(2,2,2)),partition p2 values in ((0,0,0),(1,0,0),(2,0,0),(0,0,1),(2,0,1),(1,1,1),(1,1,2),(2,2,0),(0,2,1),(1,2,1),(0,2,2)))") - // Test for single column condition. - for i := 0; i < valueNum+1; i++ { - query := fmt.Sprintf("select * from t1 where id = %v order by id,a,b", i) - tk.MustQuery(query).Check(tk1.MustQuery(query).Rows()) - query = fmt.Sprintf("select * from t1 where a = %v order by id,a,b", i) - tk.MustQuery(query).Check(tk1.MustQuery(query).Rows()) - query = fmt.Sprintf("select * from t1 where b = %v order by id,a,b", i) - tk.MustQuery(query).Check(tk1.MustQuery(query).Rows()) - } - // Test for multi-columns condition. - genCond := func() string { - col := colNames[rand.Intn(len(colNames))] - value := rand.Intn(valueNum + 2) - switch rand.Int() % 3 { - case 0: - return fmt.Sprintf(" %v = %v ", col, value) - case 1: - return fmt.Sprintf(" %v = %v ", value, col) - default: - buf := bytes.NewBuffer(nil) - buf.WriteString(fmt.Sprintf(" %v in (", col)) - n := rand.Intn(valueNum+5) + 1 - for i := 0; i < n; i++ { - if i > 0 { - buf.WriteString(",") - } - value := rand.Intn(valueNum + 2) - buf.WriteString(fmt.Sprintf("%v", value)) - } - buf.WriteString(")") - return buf.String() - } - } - for i := 0; i < 500; i++ { - condCnt := rand.Intn(condNum) + 1 - query := bytes.NewBuffer(nil) - query.WriteString("select * from t1 where ") - for j := 0; j < condCnt; j++ { - if j > 0 { - if rand.Int()%2 == 0 { - query.WriteString(" and ") - } else { - query.WriteString(" or ") - } - } - query.WriteString(genCond()) - } - query.WriteString(" order by id,a,b") - tk.MustQuery(query.String()).Check(tk1.MustQuery(query.String()).Rows()) - } + tk1 := testkit.NewTestKit(c, s.store) + tk1.MustExec("drop database if exists test_partition_1;") + tk1.MustExec("create database test_partition_1") + tk1.MustExec("use test_partition_1") + tk1.MustExec("create table t1 (id int, a int, b int)") + + inserts := []string{ + "insert into t1 (b,id,a) values (1,0,2),(2,0,2),(0,1,0),(1,1,0),(2,1,0),(0,1,1),(0,1,2),(0,2,0),(1,2,0)", + "insert into t1 (b,id,a) values (1,0,1),(0,0,2),(2,1,1),(2,1,2),(2,2,1),(1,2,2),(2,2,2)", + "insert into t1 (b,id,a) values (0,0,0),(1,0,0),(2,0,0),(0,0,1),(2,0,1),(1,1,1),(1,1,2),(2,2,0),(0,2,1),(1,2,1),(0,2,2)", + } + // prepare data. + for _, insert := range inserts { + tk.MustExec(insert) + tk1.MustExec(insert) + + // Test query without condition + query := fmt.Sprintf("select * from t1 order by id,a,b") + tk.MustQuery(query).Check(tk1.MustQuery(query).Rows()) + } + + // Test for single column condition. + for i := 0; i < valueNum+1; i++ { + query := fmt.Sprintf("select * from t1 where id = %v order by id,a,b", i) + tk.MustQuery(query).Check(tk1.MustQuery(query).Rows()) + query = fmt.Sprintf("select * from t1 where a = %v order by id,a,b", i) + tk.MustQuery(query).Check(tk1.MustQuery(query).Rows()) + query = fmt.Sprintf("select * from t1 where b = %v order by id,a,b", i) + tk.MustQuery(query).Check(tk1.MustQuery(query).Rows()) + } + // Test for multi-columns condition. + multiColumns := []string{ + "select * from t1 where 0 = a or 4 = b order by id,a,b", + "select * from t1 where b in (3,4,3,1) and b = 0 order by id,a,b", + "select * from t1 where 1 = b and id = 3 and 1 = id and b in (1,0,1,3,4,0,4,4) order by id,a,b", + "select * from t1 where 1 = b and id in (1,1,4,4,1,0,3) order by id,a,b", + "select * from t1 where 1 = b and b = 4 order by id,a,b", + } + for _, multi := range multiColumns { + tk.MustQuery(multi).Check(tk1.MustQuery(multi).Rows()) } }