From 8d9b8bda0530810621e800118d86131e35442456 Mon Sep 17 00:00:00 2001 From: Zhou Kunqin <25057648+time-and-fate@users.noreply.github.com> Date: Tue, 5 Dec 2023 21:20:49 +0800 Subject: [PATCH] This is an automated cherry-pick of #48522 Signed-off-by: ti-chi-bot --- pkg/planner/core/integration_test.go | 2174 +++++++++++ .../core/testdata/index_merge_suite_out.json | 436 +++ pkg/util/ranger/BUILD.bazel | 67 + .../r/expression/charset_and_collation.result | 1993 +++++++++++ .../r/expression/issues.result | 3167 +++++++++++++++++ .../core/issuetest/planner_issue.result | 315 ++ .../r/planner/core/range_scan_for_like.result | 1035 ++++++ .../t/planner/core/range_scan_for_like.test | 231 ++ util/ranger/checker.go | 10 - util/ranger/detacher.go | 31 +- util/ranger/points.go | 279 +- util/ranger/ranger.go | 51 +- util/ranger/ranger_test.go | 6 +- 13 files changed, 9767 insertions(+), 28 deletions(-) create mode 100644 pkg/planner/core/integration_test.go create mode 100644 pkg/planner/core/testdata/index_merge_suite_out.json create mode 100644 pkg/util/ranger/BUILD.bazel create mode 100644 tests/integrationtest/r/expression/charset_and_collation.result create mode 100644 tests/integrationtest/r/expression/issues.result create mode 100644 tests/integrationtest/r/planner/core/issuetest/planner_issue.result create mode 100644 tests/integrationtest/r/planner/core/range_scan_for_like.result create mode 100644 tests/integrationtest/t/planner/core/range_scan_for_like.test diff --git a/pkg/planner/core/integration_test.go b/pkg/planner/core/integration_test.go new file mode 100644 index 0000000000000..fbc4151911d52 --- /dev/null +++ b/pkg/planner/core/integration_test.go @@ -0,0 +1,2174 @@ +// Copyright 2019 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package core_test + +import ( + "bytes" + "fmt" + "regexp" + "strconv" + "strings" + "testing" + "time" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/pkg/config" + "github.com/pingcap/tidb/pkg/domain" + "github.com/pingcap/tidb/pkg/expression" + "github.com/pingcap/tidb/pkg/infoschema" + "github.com/pingcap/tidb/pkg/parser/auth" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/table" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/pingcap/tidb/pkg/util" + "github.com/stretchr/testify/require" +) + +func TestNoneAccessPathsFoundByIsolationRead(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int primary key)") + + tk.MustExec("select * from t") + + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + + // Don't filter mysql.SystemDB by isolation read. + tk.MustQuery("explain format = 'brief' select * from mysql.stats_meta").Check(testkit.Rows( + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tikv] table:stats_meta keep order:false, stats:pseudo")) + + _, err := tk.Exec("select * from t") + require.EqualError(t, err, "[planner:1815]Internal : No access path for table 't' is found with 'tidb_isolation_read_engines' = 'tiflash', valid values can be 'tikv'. Please check tiflash replica.") + + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash, tikv'") + tk.MustExec("select * from t") + defer config.RestoreFunc()() + config.UpdateGlobal(func(conf *config.Config) { + conf.IsolationRead.Engines = []string{"tiflash"} + }) + // Change instance config doesn't affect isolation read. + tk.MustExec("select * from t") +} + +func TestAggPushDownEngine(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int primary key, b varchar(20))") + tk.MustExec("set @@session.tidb_allow_tiflash_cop=ON") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + + tk.MustQuery("explain format = 'brief' select approx_count_distinct(a) from t").Check(testkit.Rows( + "StreamAgg 1.00 root funcs:approx_count_distinct(Column#5)->Column#3", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:approx_count_distinct(test.t.a)->Column#5", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo")) + + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tikv'") + + tk.MustQuery("explain format = 'brief' select approx_count_distinct(a) from t").Check(testkit.Rows( + "HashAgg 1.00 root funcs:approx_count_distinct(test.t.a)->Column#3", + "└─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo")) +} + +func TestIssue15110(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists crm_rd_150m") + tk.MustExec(`CREATE TABLE crm_rd_150m ( + product varchar(256) DEFAULT NULL, + uks varchar(16) DEFAULT NULL, + brand varchar(256) DEFAULT NULL, + cin varchar(16) DEFAULT NULL, + created_date timestamp NULL DEFAULT NULL, + quantity int(11) DEFAULT NULL, + amount decimal(11,0) DEFAULT NULL, + pl_date timestamp NULL DEFAULT NULL, + customer_first_date timestamp NULL DEFAULT NULL, + recent_date timestamp NULL DEFAULT NULL + ) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin;`) + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "crm_rd_150m" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("explain format = 'brief' SELECT count(*) FROM crm_rd_150m dataset_48 WHERE (CASE WHEN (month(dataset_48.customer_first_date)) <= 30 THEN '新客' ELSE NULL END) IS NOT NULL;") +} + +func TestPartitionPruningForEQ(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a datetime, b int) partition by range(weekday(a)) (partition p0 values less than(10), partition p1 values less than (100))") + + is := tk.Session().GetInfoSchema().(infoschema.InfoSchema) + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + pt := tbl.(table.PartitionedTable) + query, err := expression.ParseSimpleExprWithTableInfo(tk.Session(), "a = '2020-01-01 00:00:00'", tbl.Meta()) + require.NoError(t, err) + dbName := model.NewCIStr(tk.Session().GetSessionVars().CurrentDB) + columns, names, err := expression.ColumnInfos2ColumnsAndNames(tk.Session(), dbName, tbl.Meta().Name, tbl.Meta().Cols(), tbl.Meta()) + require.NoError(t, err) + // Even the partition is not monotonous, EQ condition should be prune! + // select * from t where a = '2020-01-01 00:00:00' + res, err := core.PartitionPruning(tk.Session(), pt, []expression.Expression{query}, nil, columns, names) + require.NoError(t, err) + require.Len(t, res, 1) + require.Equal(t, 0, res[0]) +} + +func TestNotReadOnlySQLOnTiFlash(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int, b varchar(20))") + tk.MustExec(`set @@tidb_isolation_read_engines = "tiflash"`) + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + err := tk.ExecToErr("select * from t for update") + require.EqualError(t, err, `[planner:1815]Internal : No access path for table 't' is found with 'tidb_isolation_read_engines' = 'tiflash', valid values can be 'tiflash, tikv'. Please check tiflash replica or check if the query is not readonly and sql mode is strict.`) + + err = tk.ExecToErr("insert into t select * from t") + require.EqualError(t, err, `[planner:1815]Internal : No access path for table 't' is found with 'tidb_isolation_read_engines' = 'tiflash', valid values can be 'tiflash, tikv'. Please check tiflash replica or check if the query is not readonly and sql mode is strict.`) + + tk.MustExec("prepare stmt_insert from 'insert into t select * from t where t.a = ?'") + tk.MustExec("set @a=1") + err = tk.ExecToErr("execute stmt_insert using @a") + require.EqualError(t, err, `[planner:1815]Internal : No access path for table 't' is found with 'tidb_isolation_read_engines' = 'tiflash', valid values can be 'tiflash, tikv'. Please check tiflash replica or check if the query is not readonly and sql mode is strict.`) +} + +func TestTimeToSecPushDownToTiFlash(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a time(4))") + tk.MustExec("insert into t values('700:10:10.123456')") + tk.MustExec("insert into t values('20:20:20')") + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + rows := [][]interface{}{ + {"TableReader_10", "10000.00", "root", " MppVersion: 2, data:ExchangeSender_9"}, + {"└─ExchangeSender_9", "10000.00", "mpp[tiflash]", " ExchangeType: PassThrough"}, + {" └─Projection_4", "10000.00", "mpp[tiflash]", " time_to_sec(test.t.a)->Column#3"}, + {" └─TableFullScan_8", "10000.00", "mpp[tiflash]", "table:t", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select time_to_sec(a) from t;").Check(rows) +} + +func TestRightShiftPushDownToTiFlash(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int)") + tk.MustExec("insert into t values(2147483647, 32)") + tk.MustExec("insert into t values(12, 2)") + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + rows := [][]interface{}{ + {"TableReader_10", "root", "MppVersion: 2, data:ExchangeSender_9"}, + {"└─ExchangeSender_9", "mpp[tiflash]", "ExchangeType: PassThrough"}, + {" └─Projection_4", "mpp[tiflash]", "rightshift(test.t.a, test.t.b)->Column#4"}, + {" └─TableFullScan_8", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select a >> b from t;").CheckAt([]int{0, 2, 4}, rows) +} + +func TestBitColumnPushDown(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=1") + tk.MustExec("create table t1(a bit(8), b int)") + tk.MustExec("create table t2(a bit(8), b int)") + tk.MustExec("insert into t1 values ('1', 1), ('2', 2), ('3', 3), ('4', 4), ('1', 1), ('2', 2), ('3', 3), ('4', 4)") + tk.MustExec("insert into t2 values ('1', 1), ('2', 2), ('3', 3), ('4', 4), ('1', 1), ('2', 2), ('3', 3), ('4', 4)") + sql := "select b from t1 where t1.b > (select min(t2.b) from t2 where t2.a < t1.a)" + tk.MustQuery(sql).Sort().Check(testkit.Rows("2", "2", "3", "3", "4", "4")) + rows := [][]interface{}{ + {"Projection_15", "root", "test.t1.b"}, + {"└─Apply_17", "root", "CARTESIAN inner join, other cond:gt(test.t1.b, Column#7)"}, + {" ├─TableReader_20(Build)", "root", "data:Selection_19"}, + {" │ └─Selection_19", "cop[tikv]", "not(isnull(test.t1.b))"}, + {" │ └─TableFullScan_18", "cop[tikv]", "keep order:false, stats:pseudo"}, + {" └─Selection_21(Probe)", "root", "not(isnull(Column#7))"}, + {" └─StreamAgg_23", "root", "funcs:min(test.t2.b)->Column#7"}, + {" └─TopN_24", "root", "test.t2.b, offset:0, count:1"}, + {" └─TableReader_32", "root", "data:TopN_31"}, + {" └─TopN_31", "cop[tikv]", "test.t2.b, offset:0, count:1"}, + {" └─Selection_30", "cop[tikv]", "lt(test.t2.a, test.t1.a), not(isnull(test.t2.b))"}, + {" └─TableFullScan_29", "cop[tikv]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery(fmt.Sprintf("explain analyze %s", sql)).CheckAt([]int{0, 3, 6}, rows) + tk.MustExec("insert t1 values ('A', 1);") + sql = "select a from t1 where ascii(a)=65" + tk.MustQuery(sql).Check(testkit.Rows("A")) + rows = [][]interface{}{ + {"TableReader_7", "root", "data:Selection_6"}, + {"└─Selection_6", "cop[tikv]", "eq(ascii(cast(test.t1.a, var_string(1))), 65)"}, + {" └─TableFullScan_5", "cop[tikv]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery(fmt.Sprintf("explain analyze %s", sql)).CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = `eq(concat(cast(test.t1.a, var_string(1)), "A"), "AA")` + sql = "select a from t1 where concat(a, 'A')='AA'" + tk.MustQuery(sql).Check(testkit.Rows("A")) + tk.MustQuery(fmt.Sprintf("explain analyze %s", sql)).CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = `eq(cast(test.t1.a, binary(1)), "A")` + sql = "select a from t1 where binary a='A'" + tk.MustQuery(sql).Check(testkit.Rows("A")) + tk.MustQuery(fmt.Sprintf("explain analyze %s", sql)).CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = `eq(cast(test.t1.a, var_string(1)), "A")` + sql = "select a from t1 where cast(a as char)='A'" + tk.MustQuery(sql).Check(testkit.Rows("A")) + tk.MustQuery(fmt.Sprintf("explain analyze %s", sql)).CheckAt([]int{0, 3, 6}, rows) + + tk.MustExec("insert into mysql.expr_pushdown_blacklist values('bit', 'tikv','');") + tk.MustExec("admin reload expr_pushdown_blacklist;") + rows = [][]interface{}{ + {"Selection_5", "root", `eq(cast(test.t1.a, var_string(1)), "A")`}, + {"└─TableReader_7", "root", "data:TableFullScan_6"}, + {" └─TableFullScan_6", "cop[tikv]", "keep order:false, stats:pseudo"}, + } + sql = "select a from t1 where cast(a as char)='A'" + tk.MustQuery(sql).Check(testkit.Rows("A")) + tk.MustQuery(fmt.Sprintf("explain analyze %s", sql)).CheckAt([]int{0, 3, 6}, rows) + + tk.MustExec("delete from mysql.expr_pushdown_blacklist where name='bit'") + tk.MustExec("admin reload expr_pushdown_blacklist;") + sql = "select a from t1 where ascii(a)=65" + tk.MustQuery(sql).Check(testkit.Rows("A")) + rows = [][]interface{}{ + {"TableReader_7", "root", "data:Selection_6"}, + {"└─Selection_6", "cop[tikv]", "eq(ascii(cast(test.t1.a, var_string(1))), 65)"}, + {" └─TableFullScan_5", "cop[tikv]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery(fmt.Sprintf("explain analyze %s", sql)).CheckAt([]int{0, 3, 6}, rows) + + // test collation + tk.MustExec("update mysql.tidb set VARIABLE_VALUE='True' where VARIABLE_NAME='new_collation_enabled'") + tk.MustQuery("SELECT VARIABLE_VALUE FROM mysql.tidb WHERE VARIABLE_NAME='new_collation_enabled';").Check( + testkit.Rows("True")) + tk.MustExec("create table t3 (a bit(8));") + tk.MustExec("insert into t3 values (65)") + tk.MustExec("SET NAMES utf8mb4 COLLATE utf8mb4_bin") + tk.MustQuery("select a from t3 where cast(a as char) = 'a'").Check(testkit.Rows()) + tk.MustExec("SET NAMES utf8mb4 COLLATE utf8mb4_general_ci") + tk.MustQuery("select a from t3 where cast(a as char) = 'a'").Check(testkit.Rows("A")) +} + +func TestSysdatePushDown(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(id int signed, id2 int unsigned, c varchar(11), d datetime, b double, e bit(10))") + tk.MustExec("insert into t(id, id2, c, d) values (-1, 1, 'abc', '2021-12-12')") + rows := [][]interface{}{ + {"TableReader_7", "root", "data:Selection_6"}, + {"└─Selection_6", "cop[tikv]", "gt(test.t.d, sysdate())"}, + {" └─TableFullScan_5", "cop[tikv]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where d > sysdate()"). + CheckAt([]int{0, 3, 6}, rows) + // assert sysdate isn't now after set global tidb_sysdate_is_now in the same session + tk.MustExec("set global tidb_sysdate_is_now='1'") + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where d > sysdate()"). + CheckAt([]int{0, 3, 6}, rows) + + // assert sysdate is now after set global tidb_sysdate_is_now in the new session + tk = testkit.NewTestKit(t, store) + tk.MustExec("use test") + now := time.Now() + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/expression/injectNow", fmt.Sprintf(`return(%d)`, now.Unix()))) + rows[1][2] = fmt.Sprintf("gt(test.t.d, %v)", now.Format(time.DateTime)) + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where d > sysdate()"). + CheckAt([]int{0, 3, 6}, rows) + failpoint.Disable("github.com/pingcap/tidb/pkg/expression/injectNow") + + // assert sysdate isn't now after set session tidb_sysdate_is_now false in the same session + tk.MustExec("set tidb_sysdate_is_now='0'") + rows[1][2] = "gt(test.t.d, sysdate())" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where d > sysdate()"). + CheckAt([]int{0, 3, 6}, rows) +} + +func TestTimeScalarFunctionPushDownResult(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(col1 datetime, col2 datetime, y int(8), m int(8), d int(8)) CHARSET=utf8 COLLATE=utf8_general_ci;") + tk.MustExec("insert into t values ('2022-03-24 01:02:03.040506', '9999-12-31 23:59:59', 9999, 12, 31);") + testcases := []struct { + sql string + function string + }{ + { + sql: "select col1, hour(col1) from t where hour(col1)=hour('2022-03-24 01:02:03.040506');", + function: "hour", + }, + { + sql: "select col1, month(col1) from t where month(col1)=month('2022-03-24 01:02:03.040506');", + function: "month", + }, + { + sql: "select col1, minute(col1) from t where minute(col1)=minute('2022-03-24 01:02:03.040506');", + function: "minute", + }, + { + function: "second", + sql: "select col1, second(col1) from t where second(col1)=second('2022-03-24 01:02:03.040506');", + }, + { + function: "microsecond", + sql: "select col1, microsecond(col1) from t where microsecond(col1)=microsecond('2022-03-24 01:02:03.040506');", + }, + { + function: "dayName", + sql: "select col1, dayName(col1) from t where dayName(col1)=dayName('2022-03-24 01:02:03.040506');", + }, + { + function: "dayOfMonth", + sql: "select col1, dayOfMonth(col1) from t where dayOfMonth(col1)=dayOfMonth('2022-03-24 01:02:03.040506');", + }, + { + function: "dayOfWeek", + sql: "select col1, dayOfWeek(col1) from t where dayOfWeek(col1)=dayOfWeek('2022-03-24 01:02:03.040506');", + }, + { + function: "dayOfYear", + sql: "select col1, dayOfYear(col1) from t where dayOfYear(col1)=dayOfYear('2022-03-24 01:02:03.040506');", + }, + { + function: "Date", + sql: "select col1, Date(col1) from t where Date(col1)=Date('2022-03-24 01:02:03.040506');", + }, + { + function: "Week", + sql: "select col1, Week(col1) from t where Week(col1)=Week('2022-03-24 01:02:03.040506');", + }, + { + function: "time_to_sec", + sql: "select col1, time_to_sec (col1) from t where time_to_sec(col1)=time_to_sec('2022-03-24 01:02:03.040506');", + }, + { + function: "DateDiff", + sql: "select col1, DateDiff(col1, col2) from t where DateDiff(col1, col2)=DateDiff('2022-03-24 01:02:03.040506', '9999-12-31 23:59:59');", + }, + { + function: "MonthName", + sql: "select col1, MonthName(col1) from t where MonthName(col1)=MonthName('2022-03-24 01:02:03.040506');", + }, + { + function: "MakeDate", + sql: "select col1, MakeDate(9999, 31) from t where MakeDate(y, d)=MakeDate(9999, 31);", + }, + { + function: "MakeTime", + sql: "select col1, MakeTime(12, 12, 31) from t where MakeTime(m, m, d)=MakeTime(12, 12, 31);", + }, + } + tk.MustExec("delete from mysql.expr_pushdown_blacklist where name != 'date_add'") + tk.MustExec("admin reload expr_pushdown_blacklist;") + for _, testcase := range testcases { + r1 := tk.MustQuery(testcase.sql).Rows() + tk.MustExec(fmt.Sprintf("insert into mysql.expr_pushdown_blacklist(name) values('%s');", testcase.function)) + tk.MustExec("admin reload expr_pushdown_blacklist;") + r2 := tk.MustQuery(testcase.sql).Rows() + require.EqualValues(t, r2, r1, testcase.sql) + } + tk.MustExec("delete from mysql.expr_pushdown_blacklist where name != 'date_add'") + tk.MustExec("admin reload expr_pushdown_blacklist;") +} + +func TestNumberFunctionPushDown(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(a int signed, b int unsigned,c double)") + tk.MustExec("insert into t values (-1,61,4.4)") + testcases := []struct { + sql string + function string + }{ + { + sql: "select a, mod(a,2) from t where mod(-1,2)=mod(a,2);", + function: "mod", + }, + { + sql: "select b, mod(b,2) from t where mod(61,2)=mod(b,2);", + function: "mod", + }, + { + sql: "select b,unhex(b) from t where unhex(61) = unhex(b)", + function: "unhex", + }, + { + sql: "select b, oct(b) from t where oct(61) = oct(b)", + function: "oct", + }, + { + sql: "select c, sin(c) from t where sin(4.4) = sin(c)", + function: "sin", + }, + { + sql: "select c, asin(c) from t where asin(4.4) = asin(c)", + function: "asin", + }, + { + sql: "select c, cos(c) from t where cos(4.4) = cos(c)", + function: "cos", + }, + { + sql: "select c, acos(c) from t where acos(4.4) = acos(c)", + function: "acos", + }, + { + sql: "select b,atan(b) from t where atan(61)=atan(b)", + function: "atan", + }, + { + sql: "select b, atan2(b, c) from t where atan2(61,4.4)=atan2(b,c)", + function: "atan2", + }, + { + sql: "select b,cot(b) from t where cot(61)=cot(b)", + function: "cot", + }, + { + sql: "select c from t where pi() < c", + function: "pi", + }, + } + for _, testcase := range testcases { + tk.MustExec("delete from mysql.expr_pushdown_blacklist where name != 'date_add'") + tk.MustExec("admin reload expr_pushdown_blacklist;") + r1 := tk.MustQuery(testcase.sql).Rows() + tk.MustExec(fmt.Sprintf("insert into mysql.expr_pushdown_blacklist(name) values('%s');", testcase.function)) + tk.MustExec("admin reload expr_pushdown_blacklist;") + r2 := tk.MustQuery(testcase.sql).Rows() + require.EqualValues(t, r2, r1, testcase.sql) + } +} + +func TestScalarFunctionPushDown(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(id int signed, id2 int unsigned, c varchar(11), d datetime, b double, e bit(10))") + tk.MustExec("insert into t(id, id2, c, d) values (-1, 1, '{\"a\":1}', '2021-12-12')") + rows := [][]interface{}{ + {"TableReader_7", "root", "data:Selection_6"}, + {"└─Selection_6", "cop[tikv]", "right(test.t.c, 1)"}, + {" └─TableFullScan_5", "cop[tikv]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where right(c,1);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "mod(test.t.id, test.t.id)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where mod(id, id);"). + CheckAt([]int{0, 3, 6}, rows) + rows[1][2] = "mod(test.t.id, test.t.id2)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where mod(id, id2);"). + CheckAt([]int{0, 3, 6}, rows) + rows[1][2] = "mod(test.t.id2, test.t.id)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where mod(id2, id);"). + CheckAt([]int{0, 3, 6}, rows) + rows[1][2] = "mod(test.t.id2, test.t.id2)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where mod(id2, id2);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "sin(cast(test.t.id, double BINARY))" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where sin(id);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "asin(cast(test.t.id, double BINARY))" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where asin(id);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "cos(cast(test.t.id, double BINARY))" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where cos(id);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "acos(cast(test.t.id, double BINARY))" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where acos(id);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "atan(cast(test.t.id, double BINARY))" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where atan(id);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "atan2(cast(test.t.id, double BINARY), cast(test.t.id, double BINARY))" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where atan2(id,id);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "hour(cast(test.t.d, time))" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where hour(d);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "hour(cast(test.t.d, time))" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where hour(d);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "minute(cast(test.t.d, time))" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where minute(d);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "second(cast(test.t.d, time))" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where second(d);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "month(test.t.d)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where month(d);"). + CheckAt([]int{0, 3, 6}, rows) + + //rows[1][2] = "dayname(test.t.d)" + //tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where dayname(d);"). + // CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "dayofmonth(test.t.d)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where dayofmonth(d);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "from_days(test.t.id)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where from_days(id);"). + CheckAt([]int{0, 3, 6}, rows) + + //rows[1][2] = "last_day(test.t.d)" + //tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where last_day(d);"). + // CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "gt(4, test.t.id)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where pi() > id;"). + CheckAt([]int{0, 3, 6}, rows) + + //rows[1][2] = "truncate(test.t.id, 0)" + //tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where truncate(id,0)"). + // CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "round(test.t.b)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where round(b)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "date(test.t.d)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where date(d)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "week(test.t.d)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where week(d)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "datediff(test.t.d, test.t.d)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where datediff(d,d)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "gt(test.t.d, sysdate())" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where d > sysdate()"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "ascii(cast(test.t.e, var_string(2)))" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where ascii(e);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "eq(json_valid(test.t.c), 1)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where json_valid(c)=1;"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "json_contains(cast(test.t.c, json BINARY), cast(\"1\", json BINARY))" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where json_contains(c, '1');"). + CheckAt([]int{0, 3, 6}, rows) +} + +func TestReverseUTF8PushDownToTiFlash(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a varchar(256))") + tk.MustExec("insert into t values('pingcap')") + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + rows := [][]interface{}{ + {"TableReader_10", "root", "MppVersion: 2, data:ExchangeSender_9"}, + {"└─ExchangeSender_9", "mpp[tiflash]", "ExchangeType: PassThrough"}, + {" └─Projection_4", "mpp[tiflash]", "reverse(test.t.a)->Column#3"}, + {" └─TableFullScan_8", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + } + + tk.MustQuery("explain select reverse(a) from t;").CheckAt([]int{0, 2, 4}, rows) +} + +func TestReversePushDownToTiFlash(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a binary(32))") + tk.MustExec("insert into t values('pingcap')") + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + rows := [][]interface{}{ + {"TableReader_10", "root", "MppVersion: 2, data:ExchangeSender_9"}, + {"└─ExchangeSender_9", "mpp[tiflash]", "ExchangeType: PassThrough"}, + {" └─Projection_4", "mpp[tiflash]", "reverse(test.t.a)->Column#3"}, + {" └─TableFullScan_8", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + } + + tk.MustQuery("explain select reverse(a) from t;").CheckAt([]int{0, 2, 4}, rows) +} + +func TestSpacePushDownToTiFlash(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, 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(5)") + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + rows := [][]interface{}{ + {"TableReader_10", "root", "MppVersion: 2, data:ExchangeSender_9"}, + {"└─ExchangeSender_9", "mpp[tiflash]", "ExchangeType: PassThrough"}, + {" └─Projection_4", "mpp[tiflash]", "space(test.t.a)->Column#3"}, + {" └─TableFullScan_8", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + } + + tk.MustQuery("explain select space(a) from t;").CheckAt([]int{0, 2, 4}, rows) +} + +func TestExplainAnalyzeDML2(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + cases := []struct { + prepare string + sql string + planRegexp string + }{ + // Test for alloc auto ID. + { + sql: "insert into t () values ()", + planRegexp: ".*prepare.*total.*, auto_id_allocator.*alloc_cnt: 1, Get.*num_rpc.*total_time.*commit_txn.*prewrite.*get_commit_ts.*commit.*write_keys.*, insert.*", + }, + // Test for rebase ID. + { + sql: "insert into t (a) values (99000000000)", + planRegexp: ".*prepare.*total.*, auto_id_allocator.*rebase_cnt: 1, Get.*num_rpc.*total_time.*commit_txn.*prewrite.*get_commit_ts.*commit.*write_keys.*, insert.*", + }, + // Test for alloc auto ID and rebase ID. + { + sql: "insert into t (a) values (null), (99000000000)", + planRegexp: ".*prepare.*total.*, auto_id_allocator.*alloc_cnt: 1, rebase_cnt: 1, Get.*num_rpc.*total_time.*commit_txn.*prewrite.*get_commit_ts.*commit.*write_keys.*, insert.*", + }, + // Test for insert ignore. + { + sql: "insert ignore into t values (null,1), (2, 2), (99000000000, 3), (100000000000, 4)", + planRegexp: ".*prepare.*total.*, auto_id_allocator.*alloc_cnt: 1, rebase_cnt: 2, Get.*num_rpc.*total_time.*commit_txn.*count: 3, prewrite.*get_commit_ts.*commit.*write_keys.*, check_insert.*", + }, + // Test for insert on duplicate. + { + sql: "insert into t values (null,null), (1,1),(2,2) on duplicate key update a = a + 100000000000", + planRegexp: ".*prepare.*total.*, auto_id_allocator.*alloc_cnt: 1, rebase_cnt: 1, Get.*num_rpc.*total_time.*commit_txn.*count: 2, prewrite.*get_commit_ts.*commit.*write_keys.*, check_insert.*", + }, + // Test for replace with alloc ID. + { + sql: "replace into t () values ()", + planRegexp: ".*auto_id_allocator.*alloc_cnt: 1, Get.*num_rpc.*total_time.*commit_txn.*prewrite.*get_commit_ts.*commit.*write_keys.*", + }, + // Test for replace with alloc ID and rebase ID. + { + sql: "replace into t (a) values (null), (99000000000)", + planRegexp: ".*auto_id_allocator.*alloc_cnt: 1, rebase_cnt: 1, Get.*num_rpc.*total_time.*commit_txn.*prewrite.*get_commit_ts.*commit.*write_keys.*", + }, + // Test for update with rebase ID. + { + prepare: "insert into t values (1,1),(2,2)", + sql: "update t set a=a*100000000000", + planRegexp: ".*auto_id_allocator.*rebase_cnt: 2, Get.*num_rpc.*total_time.*commit_txn.*prewrite.*get_commit_ts.*commit.*write_keys.*", + }, + } + + for _, ca := range cases { + for i := 0; i < 3; i++ { + tk.MustExec("drop table if exists t") + switch i { + case 0: + tk.MustExec("create table t (a bigint auto_increment, b int, primary key (a));") + case 1: + tk.MustExec("create table t (a bigint unsigned auto_increment, b int, primary key (a));") + case 2: + if strings.Contains(ca.sql, "on duplicate key") { + continue + } + tk.MustExec("create table t (a bigint primary key auto_random(5), b int);") + tk.MustExec("set @@allow_auto_random_explicit_insert=1;") + default: + panic("should never happen") + } + if ca.prepare != "" { + tk.MustExec(ca.prepare) + } + res := tk.MustQuery("explain analyze " + ca.sql) + resBuff := bytes.NewBufferString("") + for _, row := range res.Rows() { + _, _ = fmt.Fprintf(resBuff, "%s\t", row) + } + explain := resBuff.String() + require.Regexpf(t, ca.planRegexp, explain, "idx: %v,sql: %v", i, ca.sql) + } + } + + // Test for table without auto id. + for _, ca := range cases { + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a bigint, b int);") + tk.MustExec("insert into t () values ()") + if ca.prepare != "" { + tk.MustExec(ca.prepare) + } + res := tk.MustQuery("explain analyze " + ca.sql) + resBuff := bytes.NewBufferString("") + for _, row := range res.Rows() { + _, _ = fmt.Fprintf(resBuff, "%s\t", row) + } + explain := resBuff.String() + require.NotContainsf(t, explain, "auto_id_allocator", "sql: %v, explain: %v", ca.sql, explain) + } +} + +func TestConflictReadFromStorage(t *testing.T) { + failpoint.Enable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune", `return(true)`) + defer failpoint.Disable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune") + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec(`create table t ( + a int, b int, c varchar(20), + primary key(a), key(b), key(c) + ) partition by range columns(a) ( + partition p0 values less than(6), + partition p1 values less than(11), + partition p2 values less than(16));`) + tk.MustExec(`insert into t values (1,1,"1"), (2,2,"2"), (8,8,"8"), (11,11,"11"), (15,15,"15")`) + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + tk.MustQuery(`explain select /*+ read_from_storage(tikv[t partition(p0)], tiflash[t partition(p1, p2)]) */ * from t`) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 Storage hints are conflict, you can only specify one storage type of table test.t")) + tk.MustQuery(`explain select /*+ read_from_storage(tikv[t], tiflash[t]) */ * from t`) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 Storage hints are conflict, you can only specify one storage type of table test.t")) +} + +func TestIssue29503(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + defer config.RestoreFunc()() + config.UpdateGlobal(func(conf *config.Config) { + conf.Status.RecordQPSbyDB = true + }) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int);") + require.NoError(t, tk.ExecToErr("create binding for select 1 using select 1;")) + require.NoError(t, tk.ExecToErr("create binding for select a from t using select a from t;")) + res := tk.MustQuery("show session bindings;") + require.Len(t, res.Rows(), 2) +} + +func TestIssue31202(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("create table t31202(a int primary key, b int);") + + tbl, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t31202", L: "t31202"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + + tk.MustQuery("explain format = 'brief' select * from t31202;").Check(testkit.Rows( + "TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 mpp[tiflash] table:t31202 keep order:false, stats:pseudo")) + + tk.MustQuery("explain format = 'brief' select * from t31202 use index (primary);").Check(testkit.Rows( + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tikv] table:t31202 keep order:false, stats:pseudo")) + tk.MustExec("drop table if exists t31202") +} + +func TestAggPushToCopForCachedTable(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec(`create table t32157( + process_code varchar(8) NOT NULL, + ctrl_class varchar(2) NOT NULL, + ctrl_type varchar(1) NOT NULL, + oper_no varchar(12) DEFAULT NULL, + modify_date datetime DEFAULT NULL, + d_c_flag varchar(2) NOT NULL, + PRIMARY KEY (process_code,ctrl_class,d_c_flag) NONCLUSTERED);`) + tk.MustExec("insert into t32157 values ('GDEP0071', '05', '1', '10000', '2016-06-29 00:00:00', 'C')") + tk.MustExec("insert into t32157 values ('GDEP0071', '05', '0', '0000', '2016-06-01 00:00:00', 'D')") + tk.MustExec("alter table t32157 cache") + + tk.MustQuery("explain format = 'brief' select /*+AGG_TO_COP()*/ count(*) from t32157 ignore index(primary) where process_code = 'GDEP0071'").Check(testkit.Rows( + "StreamAgg 1.00 root funcs:count(1)->Column#8]\n" + + "[└─UnionScan 10.00 root eq(test.t32157.process_code, \"GDEP0071\")]\n" + + "[ └─TableReader 10.00 root data:Selection]\n" + + "[ └─Selection 10.00 cop[tikv] eq(test.t32157.process_code, \"GDEP0071\")]\n" + + "[ └─TableFullScan 10000.00 cop[tikv] table:t32157 keep order:false, stats:pseudo")) + + require.Eventually(t, func() bool { + tk.MustQuery("select /*+AGG_TO_COP()*/ count(*) from t32157 ignore index(primary) where process_code = 'GDEP0071'").Check(testkit.Rows("2")) + return tk.Session().GetSessionVars().StmtCtx.ReadFromTableCache + }, 10*time.Second, 500*time.Millisecond) + + tk.MustExec("drop table if exists t31202") +} + +func TestTiFlashFineGrainedShuffleWithMaxTiFlashThreads(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@tidb_enforce_mpp = on") + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1(c1 int, c2 int)") + tbl1, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t1", L: "t1"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl1.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + + sql := "explain select row_number() over w1 from t1 window w1 as (partition by c1);" + + getStreamCountFromExplain := func(rows [][]interface{}) (res []uint64) { + re := regexp.MustCompile("stream_count: ([0-9]+)") + for _, row := range rows { + buf := bytes.NewBufferString("") + _, _ = fmt.Fprintf(buf, "%s\n", row) + if matched := re.FindStringSubmatch(buf.String()); matched != nil { + require.Equal(t, len(matched), 2) + c, err := strconv.ParseUint(matched[1], 10, 64) + require.NoError(t, err) + res = append(res, c) + } + } + return res + } + + // tiflash_fine_grained_shuffle_stream_count should be same with tidb_max_tiflash_threads. + tk.MustExec("set @@tiflash_fine_grained_shuffle_stream_count = 0") + tk.MustExec("set @@tidb_max_tiflash_threads = 10") + rows := tk.MustQuery(sql).Rows() + streamCount := getStreamCountFromExplain(rows) + // require.Equal(t, len(streamCount), 1) + require.Equal(t, uint64(10), streamCount[0]) + + // tiflash_fine_grained_shuffle_stream_count should be default value when tidb_max_tiflash_threads is -1. + tk.MustExec("set @@tiflash_fine_grained_shuffle_stream_count = 0") + tk.MustExec("set @@tidb_max_tiflash_threads = -1") + rows = tk.MustQuery(sql).Rows() + streamCount = getStreamCountFromExplain(rows) + // require.Equal(t, len(streamCount), 1) + require.Equal(t, uint64(variable.DefStreamCountWhenMaxThreadsNotSet), streamCount[0]) + + // tiflash_fine_grained_shuffle_stream_count should be default value when tidb_max_tiflash_threads is 0. + tk.MustExec("set @@tiflash_fine_grained_shuffle_stream_count = 0") + tk.MustExec("set @@tidb_max_tiflash_threads = 0") + rows = tk.MustQuery(sql).Rows() + streamCount = getStreamCountFromExplain(rows) + // require.Equal(t, len(streamCount), 1) + require.Equal(t, uint64(variable.DefStreamCountWhenMaxThreadsNotSet), streamCount[0]) + + // Disabled when tiflash_fine_grained_shuffle_stream_count is -1. + tk.MustExec("set @@tiflash_fine_grained_shuffle_stream_count = -1") + tk.MustExec("set @@tidb_max_tiflash_threads = 10") + rows = tk.MustQuery(sql).Rows() + streamCount = getStreamCountFromExplain(rows) + require.Equal(t, len(streamCount), 0) + + // Test when tiflash_fine_grained_shuffle_stream_count is greater than 0. + tk.MustExec("set @@tiflash_fine_grained_shuffle_stream_count = 16") + tk.MustExec("set @@tidb_max_tiflash_threads = 10") + rows = tk.MustQuery(sql).Rows() + streamCount = getStreamCountFromExplain(rows) + // require.Equal(t, len(streamCount), 1) + require.Equal(t, uint64(16), streamCount[0]) +} + +func TestIssue33175(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec("create table t (id bigint(45) unsigned not null, c varchar(20), primary key(id));") + tk.MustExec("insert into t values (9734095886065816707, 'a'), (10353107668348738101, 'b'), (0, 'c');") + tk.MustExec("begin") + tk.MustExec("insert into t values (33, 'd');") + tk.MustQuery("select max(id) from t;").Check(testkit.Rows("10353107668348738101")) + tk.MustExec("rollback") + + tk.MustExec("alter table t cache") + for { + tk.MustQuery("select max(id) from t;").Check(testkit.Rows("10353107668348738101")) + if tk.Session().GetSessionVars().StmtCtx.ReadFromTableCache { + break + } + } + + // // With subquery, like the original issue case. + for { + tk.MustQuery("select * from t where id > (select max(id) from t where t.id > 0);").Check(testkit.Rows()) + if tk.Session().GetSessionVars().StmtCtx.ReadFromTableCache { + break + } + } + + // Test order by desc / asc. + tk.MustQuery("select id from t order by id desc;").Check(testkit.Rows( + "10353107668348738101", + "9734095886065816707", + "0")) + + tk.MustQuery("select id from t order by id asc;").Check(testkit.Rows( + "0", + "9734095886065816707", + "10353107668348738101")) + + tk.MustExec("alter table t nocache") + tk.MustExec("drop table t") + + // Cover more code that use union scan + // TableReader/IndexReader/IndexLookup + for idx, q := range []string{ + "create temporary table t (id bigint unsigned, c int default null, index(id))", + "create temporary table t (id bigint unsigned primary key)", + } { + tk.MustExec(q) + tk.MustExec("insert into t(id) values (1), (3), (9734095886065816707), (9734095886065816708)") + tk.MustQuery("select min(id) from t").Check(testkit.Rows("1")) + tk.MustQuery("select max(id) from t").Check(testkit.Rows("9734095886065816708")) + tk.MustQuery("select id from t order by id asc").Check(testkit.Rows( + "1", "3", "9734095886065816707", "9734095886065816708")) + tk.MustQuery("select id from t order by id desc").Check(testkit.Rows( + "9734095886065816708", "9734095886065816707", "3", "1")) + if idx == 0 { + tk.MustQuery("select * from t order by id asc").Check(testkit.Rows( + "1 ", + "3 ", + "9734095886065816707 ", + "9734095886065816708 ")) + tk.MustQuery("select * from t order by id desc").Check(testkit.Rows( + "9734095886065816708 ", + "9734095886065816707 ", + "3 ", + "1 ")) + } + tk.MustExec("drop table t") + } + + // More and more test + tk.MustExec("create global temporary table `tmp1` (id bigint unsigned primary key) on commit delete rows;") + tk.MustExec("begin") + tk.MustExec("insert into tmp1 values (0),(1),(2),(65536),(9734095886065816707),(9734095886065816708);") + tk.MustQuery("select * from tmp1 where id <= 65534 or (id > 65535 and id < 9734095886065816700) or id >= 9734095886065816707 order by id desc;").Check(testkit.Rows( + "9734095886065816708", "9734095886065816707", "65536", "2", "1", "0")) + + tk.MustQuery("select * from tmp1 where id <= 65534 or (id > 65535 and id < 9734095886065816700) or id >= 9734095886065816707 order by id asc;").Check(testkit.Rows( + "0", "1", "2", "65536", "9734095886065816707", "9734095886065816708")) + + tk.MustExec("create global temporary table `tmp2` (id bigint primary key) on commit delete rows;") + tk.MustExec("begin") + tk.MustExec("insert into tmp2 values(-2),(-1),(0),(1),(2);") + tk.MustQuery("select * from tmp2 where id <= -1 or id > 0 order by id desc;").Check(testkit.Rows("2", "1", "-1", "-2")) + tk.MustQuery("select * from tmp2 where id <= -1 or id > 0 order by id asc;").Check(testkit.Rows("-2", "-1", "1", "2")) +} + +func TestIssue35083(t *testing.T) { + defer func() { + variable.SetSysVar(variable.TiDBOptProjectionPushDown, variable.BoolToOnOff(config.GetGlobalConfig().Performance.ProjectionPushDown)) + }() + defer config.RestoreFunc()() + config.UpdateGlobal(func(conf *config.Config) { + conf.Performance.ProjectionPushDown = true + }) + variable.SetSysVar(variable.TiDBOptProjectionPushDown, variable.BoolToOnOff(config.GetGlobalConfig().Performance.ProjectionPushDown)) + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t1 (a varchar(100), b int)") + tk.MustQuery("select @@tidb_opt_projection_push_down").Check(testkit.Rows("1")) + tk.MustQuery("explain format = 'brief' select cast(a as datetime) from t1").Check(testkit.Rows( + "TableReader 10000.00 root data:Projection", + "└─Projection 10000.00 cop[tikv] cast(test.t1.a, datetime BINARY)->Column#4", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo")) +} + +func TestRepeatPushDownToTiFlash(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int)") + tk.MustExec("insert into t values(2147483647, 2)") + tk.MustExec("insert into t values(12, 2)") + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + rows := [][]interface{}{ + {"TableReader_10", "root", "MppVersion: 2, data:ExchangeSender_9"}, + {"└─ExchangeSender_9", "mpp[tiflash]", "ExchangeType: PassThrough"}, + {" └─Projection_4", "mpp[tiflash]", "repeat(cast(test.t.a, var_string(20)), test.t.b)->Column#4"}, + {" └─TableFullScan_8", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select repeat(a,b) from t;").CheckAt([]int{0, 2, 4}, rows) +} + +func TestIssue36194(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int)") + // create virtual tiflash replica. + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + tk.MustQuery("explain format = 'brief' select /*+ read_from_storage(tiflash[t]) */ * from t where a + 1 > 20 limit 100;;").Check(testkit.Rows( + "Limit 100.00 root offset:0, count:100", + "└─TableReader 100.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 100.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Limit 100.00 mpp[tiflash] offset:0, count:100", + " └─Selection 100.00 mpp[tiflash] gt(plus(test.t.a, 1), 20)", + " └─TableFullScan 125.00 mpp[tiflash] table:t pushed down filter:empty, keep order:false, stats:pseudo")) +} + +func TestGetFormatPushDownToTiFlash(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("create table t(location varchar(10));") + tk.MustExec("insert into t values('USA'), ('JIS'), ('ISO'), ('EUR'), ('INTERNAL')") + tk.MustExec("set @@tidb_enforce_mpp=1;") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash';") + + tbl, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t", L: "t"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + + tk.MustQuery("explain format = 'brief' select GET_FORMAT(DATE, location) from t;").Check(testkit.Rows( + "TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 10000.00 mpp[tiflash] get_format(DATE, test.t.location)->Column#3", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo")) +} + +func TestAggWithJsonPushDownToTiFlash(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a json);") + tk.MustExec("insert into t values(null);") + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.tidb_allow_tiflash_cop=ON") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + rows := [][]interface{}{ + {"HashAgg_6", "root", "funcs:avg(Column#4)->Column#3"}, + {"└─Projection_19", "root", "cast(test.t.a, double BINARY)->Column#4"}, + {" └─TableReader_12", "root", "data:TableFullScan_11"}, + {" └─TableFullScan_11", "cop[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select avg(a) from t;").CheckAt([]int{0, 2, 4}, rows) + + rows = [][]interface{}{ + {"HashAgg_6", "root", "funcs:sum(Column#4)->Column#3"}, + {"└─Projection_19", "root", "cast(test.t.a, double BINARY)->Column#4"}, + {" └─TableReader_12", "root", "data:TableFullScan_11"}, + {" └─TableFullScan_11", "cop[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select sum(a) from t;").CheckAt([]int{0, 2, 4}, rows) + + rows = [][]interface{}{ + {"HashAgg_6", "root", "funcs:group_concat(Column#4 separator \",\")->Column#3"}, + {"└─Projection_13", "root", "cast(test.t.a, var_string(4294967295))->Column#4"}, + {" └─TableReader_10", "root", "data:TableFullScan_9"}, + {" └─TableFullScan_9", "cop[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select /*+ hash_agg() */ group_concat(a) from t;").CheckAt([]int{0, 2, 4}, rows) +} + +func TestLeftShiftPushDownToTiFlash(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int)") + tk.MustExec("insert into t values(2147483647, 32)") + tk.MustExec("insert into t values(12, 2)") + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + + // Create virtual tiflash replica info. + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + rows := [][]interface{}{ + {"TableReader_10", "root", "MppVersion: 2, data:ExchangeSender_9"}, + {"└─ExchangeSender_9", "mpp[tiflash]", "ExchangeType: PassThrough"}, + {" └─Projection_4", "mpp[tiflash]", "leftshift(test.t.a, test.t.b)->Column#4"}, + {" └─TableFullScan_8", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select a << b from t;").CheckAt([]int{0, 2, 4}, rows) +} + +func TestHexIntOrStrPushDownToTiFlash(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b varchar(10));") + tk.MustExec("insert into t values(1, 'tiflash');") + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + + tbl, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t", L: "t"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + + rows := [][]interface{}{ + {"TableReader_10", "root", "MppVersion: 2, data:ExchangeSender_9"}, + {"└─ExchangeSender_9", "mpp[tiflash]", "ExchangeType: PassThrough"}, + {" └─Projection_4", "mpp[tiflash]", "hex(test.t.a)->Column#4"}, + {" └─TableFullScan_8", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select hex(a) from t;").CheckAt([]int{0, 2, 4}, rows) + + rows = [][]interface{}{ + {"TableReader_10", "root", "MppVersion: 2, data:ExchangeSender_9"}, + {"└─ExchangeSender_9", "mpp[tiflash]", "ExchangeType: PassThrough"}, + {" └─Projection_4", "mpp[tiflash]", "hex(test.t.b)->Column#4"}, + {" └─TableFullScan_8", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select hex(b) from t;").CheckAt([]int{0, 2, 4}, rows) +} + +func TestBinPushDownToTiFlash(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, 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.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + + tbl, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t", L: "t"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + + rows := [][]interface{}{ + {"TableReader_10", "root", "MppVersion: 2, data:ExchangeSender_9"}, + {"└─ExchangeSender_9", "mpp[tiflash]", "ExchangeType: PassThrough"}, + {" └─Projection_4", "mpp[tiflash]", "bin(test.t.a)->Column#3"}, + {" └─TableFullScan_8", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select bin(a) from t;").CheckAt([]int{0, 2, 4}, rows) +} + +func TestEltPushDownToTiFlash(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b varchar(20))") + tk.MustExec("insert into t values(2147483647, '32')") + tk.MustExec("insert into t values(12, 'abc')") + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + + // Create virtual tiflash replica info. + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + rows := [][]interface{}{ + {"TableReader_10", "root", "MppVersion: 2, data:ExchangeSender_9"}, + {"└─ExchangeSender_9", "mpp[tiflash]", "ExchangeType: PassThrough"}, + {" └─Projection_4", "mpp[tiflash]", "elt(test.t.a, test.t.b)->Column#4"}, + {" └─TableFullScan_8", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select elt(a, b) from t;").CheckAt([]int{0, 2, 4}, rows) +} + +func TestRegexpInstrPushDownToTiFlash(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists test.t;") + tk.MustExec("create table test.t (expr varchar(30), pattern varchar(30), pos int, occur int, ret_op int, match_type varchar(30));") + tk.MustExec("insert into test.t values ('123', '12.', 1, 1, 0, ''), ('aBb', 'bb', 1, 1, 0, 'i'), ('ab\nabc', '^abc$', 1, 1, 0, 'm');") + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + + // Create virtual tiflash replica info. + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + rows := [][]interface{}{ + {"TableReader_10", "root", "MppVersion: 2, data:ExchangeSender_9"}, + {"└─ExchangeSender_9", "mpp[tiflash]", "ExchangeType: PassThrough"}, + {" └─Projection_4", "mpp[tiflash]", "regexp_instr(test.t.expr, test.t.pattern, 1, 1, 0, test.t.match_type)->Column#8"}, + {" └─TableFullScan_8", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select regexp_instr(expr, pattern, 1, 1, 0, match_type) as res from test.t;").CheckAt([]int{0, 2, 4}, rows) +} + +func TestRegexpSubstrPushDownToTiFlash(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists test.t;") + tk.MustExec("create table test.t (expr varchar(30), pattern varchar(30), pos int, occur int, match_type varchar(30));") + tk.MustExec("insert into test.t values ('123', '12.', 1, 1, ''), ('aBb', 'bb', 1, 1, 'i'), ('ab\nabc', '^abc$', 1, 1, 'm');") + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + + // Create virtual tiflash replica info. + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + rows := [][]interface{}{ + {"TableReader_10", "root", "MppVersion: 2, data:ExchangeSender_9"}, + {"└─ExchangeSender_9", "mpp[tiflash]", "ExchangeType: PassThrough"}, + {" └─Projection_4", "mpp[tiflash]", "regexp_substr(test.t.expr, test.t.pattern, 1, 1, test.t.match_type)->Column#7"}, + {" └─TableFullScan_8", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select regexp_substr(expr, pattern, 1, 1, match_type) as res from test.t;").CheckAt([]int{0, 2, 4}, rows) +} + +func TestRegexpReplacePushDownToTiFlash(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists test.t;") + tk.MustExec("create table test.t (expr varchar(30), pattern varchar(30), repl varchar(30), pos int, occur int, match_type varchar(30));") + tk.MustExec("insert into test.t values ('123', '12.', '233', 1, 1, ''), ('aBb', 'bb', 'bc', 1, 1, 'i'), ('ab\nabc', '^abc$', 'd', 1, 1, 'm');") + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + + // Create virtual tiflash replica info. + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + rows := [][]interface{}{ + {"TableReader_10", "root", "MppVersion: 2, data:ExchangeSender_9"}, + {"└─ExchangeSender_9", "mpp[tiflash]", "ExchangeType: PassThrough"}, + {" └─Projection_4", "mpp[tiflash]", "regexp_replace(test.t.expr, test.t.pattern, test.t.repl, 1, 1, test.t.match_type)->Column#8"}, + {" └─TableFullScan_8", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select regexp_replace(expr, pattern, repl, 1, 1, match_type) as res from test.t;").CheckAt([]int{0, 2, 4}, rows) +} + +func TestCastTimeAsDurationToTiFlash(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a date, b datetime(4))") + tk.MustExec("insert into t values('2021-10-26', '2021-10-26')") + tk.MustExec("insert into t values('2021-10-26', '2021-10-26 11:11:11')") + tk.MustExec("insert into t values('2021-10-26', '2021-10-26 11:11:11.111111')") + tk.MustExec("insert into t values('2021-10-26', '2021-10-26 11:11:11.123456')") + tk.MustExec("insert into t values('2021-10-26', '2021-10-26 11:11:11.999999')") + + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + + // Create virtual tiflash replica info. + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + rows := [][]interface{}{ + {"TableReader_10", "root", "MppVersion: 2, data:ExchangeSender_9"}, + {"└─ExchangeSender_9", "mpp[tiflash]", "ExchangeType: PassThrough"}, + {" └─Projection_4", "mpp[tiflash]", "cast(test.t.a, time BINARY)->Column#4, cast(test.t.b, time BINARY)->Column#5"}, + {" └─TableFullScan_8", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select cast(a as time), cast(b as time) from t;").CheckAt([]int{0, 2, 4}, rows) +} + +func TestUnhexPushDownToTiFlash(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b varchar(20));") + tk.MustExec("insert into t values(6162, '7469666C617368');") + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + + tbl, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t", L: "t"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + + rows := [][]interface{}{ + {"TableReader_10", "root", "MppVersion: 2, data:ExchangeSender_9"}, + {"└─ExchangeSender_9", "mpp[tiflash]", "ExchangeType: PassThrough"}, + {" └─Projection_4", "mpp[tiflash]", "unhex(cast(test.t.a, var_string(20)))->Column#4"}, + {" └─TableFullScan_8", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select unhex(a) from t;").CheckAt([]int{0, 2, 4}, rows) + + rows = [][]interface{}{ + {"TableReader_10", "root", "MppVersion: 2, data:ExchangeSender_9"}, + {"└─ExchangeSender_9", "mpp[tiflash]", "ExchangeType: PassThrough"}, + {" └─Projection_4", "mpp[tiflash]", "unhex(test.t.b)->Column#4"}, + {" └─TableFullScan_8", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select unhex(b) from t;").CheckAt([]int{0, 2, 4}, rows) +} + +func TestLeastGretestStringPushDownToTiFlash(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a varchar(20), b varchar(20))") + tk.MustExec("insert into t values('123', '234')") + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + + tbl, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t", L: "t"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + + rows := [][]interface{}{ + {"TableReader_10", "root", "MppVersion: 2, data:ExchangeSender_9"}, + {"└─ExchangeSender_9", "mpp[tiflash]", "ExchangeType: PassThrough"}, + {" └─Projection_4", "mpp[tiflash]", "least(test.t.a, test.t.b)->Column#4"}, + {" └─TableFullScan_8", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select least(a, b) from t;").CheckAt([]int{0, 2, 4}, rows) + + rows = [][]interface{}{ + {"TableReader_10", "root", "MppVersion: 2, data:ExchangeSender_9"}, + {"└─ExchangeSender_9", "mpp[tiflash]", "ExchangeType: PassThrough"}, + {" └─Projection_4", "mpp[tiflash]", "greatest(test.t.a, test.t.b)->Column#4"}, + {" └─TableFullScan_8", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select greatest(a, b) from t;").CheckAt([]int{0, 2, 4}, rows) +} + +func TestTiFlashReadForWriteStmt(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int)") + tk.MustExec("insert into t values(1, 2)") + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t2(a int)") + tk.MustExec("set @@tidb_allow_mpp=1") + + // Default should be 1 + tk.MustQuery("select @@tidb_enable_tiflash_read_for_write_stmt").Check(testkit.Rows("1")) + // Set ON + tk.MustExec("set @@tidb_enable_tiflash_read_for_write_stmt = ON") + tk.MustQuery("show warnings").Check(testkit.Rows()) + tk.MustQuery("select @@tidb_enable_tiflash_read_for_write_stmt").Check(testkit.Rows("1")) + // Set OFF + tk.MustExec("set @@tidb_enable_tiflash_read_for_write_stmt = OFF") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 tidb_enable_tiflash_read_for_write_stmt is always turned on. This variable has been deprecated and will be removed in the future releases")) + tk.MustQuery("select @@tidb_enable_tiflash_read_for_write_stmt").Check(testkit.Rows("1")) + + tbl, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t", L: "t"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + + tbl2, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t2", L: "t2"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl2.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + + checkRes := func(r [][]interface{}, pos int, expected string) { + check := false + for i := range r { + if r[i][pos] == expected { + check = true + break + } + } + require.Equal(t, check, true) + } + + check := func(query string) { + // If sql mode is strict, read does not push down to tiflash + tk.MustExec("set @@sql_mode = 'strict_trans_tables'") + tk.MustExec("set @@tidb_enforce_mpp=0") + rs := tk.MustQuery(query).Rows() + checkRes(rs, 2, "cop[tikv]") + tk.MustQuery("show warnings").Check(testkit.Rows()) + + // If sql mode is strict and tidb_enforce_mpp is on, read does not push down to tiflash + // and should return a warning. + tk.MustExec("set @@tidb_enforce_mpp=1") + rs = tk.MustQuery(query).Rows() + checkRes(rs, 2, "cop[tikv]") + rs = tk.MustQuery("show warnings").Rows() + checkRes(rs, 2, "MPP mode may be blocked because the query is not readonly and sql mode is strict.") + + // If sql mode is not strict, read should push down to tiflash + tk.MustExec("set @@sql_mode = ''") + rs = tk.MustQuery(query).Rows() + checkRes(rs, 2, "mpp[tiflash]") + tk.MustQuery("show warnings").Check(testkit.Rows()) + } + + // Insert into ... select + check("explain insert into t2 select a+b from t") + check("explain insert into t2 select t.a from t2 join t on t2.a = t.a") + + // Replace into ... select + check("explain replace into t2 select a+b from t") + + // CTE + check("explain update t set a=a+1 where b in (select a from t2 where t.a > t2.a)") +} + +func TestPointGetWithSelectLock(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(a int, b int, primary key(a, b));") + tk.MustExec("create table t1(c int unique, d int);") + tbl, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t", L: "t"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + tbl1, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t1", L: "t1"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl1.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + + sqls := []string{ + "explain select a, b from t where (a = 1 and b = 2) or (a =2 and b = 1) for update;", + "explain select a, b from t where a = 1 and b = 2 for update;", + "explain select c, d from t1 where c = 1 for update;", + "explain select c, d from t1 where c = 1 and d = 1 for update;", + "explain select c, d from t1 where (c = 1 or c = 2 )and d = 1 for update;", + "explain select c, d from t1 where c in (1,2,3,4) for update;", + } + tk.MustExec("set @@tidb_enable_tiflash_read_for_write_stmt = on;") + tk.MustExec("set @@sql_mode='';") + tk.MustExec("set @@tidb_isolation_read_engines='tidb,tiflash';") + tk.MustExec("begin;") + // assert point get / batch point get can't work with tiflash in interaction txn + for _, sql := range sqls { + err = tk.ExecToErr(sql) + require.Error(t, err) + } + // assert point get / batch point get can work with tikv in interaction txn + tk.MustExec("set @@tidb_isolation_read_engines='tidb,tikv,tiflash';") + for _, sql := range sqls { + tk.MustQuery(sql) + } + tk.MustExec("commit") + // assert point get / batch point get can work with tiflash in auto commit + tk.MustExec("set @@tidb_isolation_read_engines='tidb,tiflash';") + for _, sql := range sqls { + tk.MustQuery(sql) + } +} + +func TestPlanCacheForIndexRangeFallback(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec(`set @@tidb_enable_prepared_plan_cache=1`) + tk.MustExec("set @@tidb_enable_collect_execution_info=0") // In this way `explain for connection id` doesn't display execution info. + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a varchar(10), b varchar(10), c varchar(10), index idx_a_b(a, b))") + tk.MustExec("set @@tidb_opt_range_max_size=1330") // 1330 is the memory usage of ["aa","aa"], ["bb","bb"], ["cc","cc"], ["dd","dd"], ["ee","ee"]. + rows := tk.MustQuery("explain format='brief' select * from t where a in ('aa', 'bb', 'cc', 'dd', 'ee')").Rows() + require.True(t, strings.Contains(rows[1][0].(string), "IndexRangeScan")) + require.True(t, strings.Contains(rows[1][4].(string), "range:[\"aa\",\"aa\"], [\"bb\",\"bb\"], [\"cc\",\"cc\"], [\"dd\",\"dd\"], [\"ee\",\"ee\"]")) + rows = tk.MustQuery("explain format='brief' select * from t where a in ('aaaaaaaaaa', 'bbbbbbbbbb', 'cccccccccc', 'dddddddddd', 'eeeeeeeeee')").Rows() + // 1330 is not enough for ["aaaaaaaaaa","aaaaaaaaaa"], ["bbbbbbbbbb","bbbbbbbbbb"], ["cccccccccc","cccccccccc"], ["dddddddddd","dddddddddd"], ["eeeeeeeeee","eeeeeeeeee"]. + // So it falls back to table full scan. + require.True(t, strings.Contains(rows[2][0].(string), "TableFullScan")) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 Memory capacity of 1330 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen")) + + // Test rebuilding ranges for the cached plan doesn't have memory limit. + tk.MustExec("prepare stmt1 from 'select * from t where a in (?, ?, ?, ?, ?)'") + tk.MustExec("set @a='aa', @b='bb', @c='cc', @d='dd', @e='ee'") + tk.MustExec("execute stmt1 using @a, @b, @c, @d, @e") + tk.MustQuery("show warnings").Check(testkit.Rows()) // Range fallback doesn't happen and the plan can be put into cache. + tk.MustExec("set @a='aaaaaaaaaa', @b='bbbbbbbbbb', @c='cccccccccc', @d='dddddddddd', @e='eeeeeeeeee'") + tk.MustExec("execute stmt1 using @a, @b, @c, @d, @e") + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + tk.MustExec("execute stmt1 using @a, @b, @c, @d, @e") + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + rows = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() + // We don't limit range mem usage when rebuilding ranges for the cached plan. + // So ["aaaaaaaaaa","aaaaaaaaaa"], ["bbbbbbbbbb","bbbbbbbbbb"], ["cccccccccc","cccccccccc"], ["dddddddddd","dddddddddd"], ["eeeeeeeeee","eeeeeeeeee"] can still be built even if its mem usage exceeds 1330. + require.True(t, strings.Contains(rows[1][0].(string), "IndexRangeScan")) + require.True(t, strings.Contains(rows[1][4].(string), "range:[\"aaaaaaaaaa\",\"aaaaaaaaaa\"], [\"bbbbbbbbbb\",\"bbbbbbbbbb\"], [\"cccccccccc\",\"cccccccccc\"], [\"dddddddddd\",\"dddddddddd\"], [\"eeeeeeeeee\",\"eeeeeeeeee\"]")) + + // Test the plan with range fallback would not be put into cache. + tk.MustExec("prepare stmt2 from 'select * from t where a in (?, ?, ?, ?, ?) and b in (?, ?, ?, ?, ?)'") + tk.MustExec("set @a='aa', @b='bb', @c='cc', @d='dd', @e='ee', @f='ff', @g='gg', @h='hh', @i='ii', @j='jj'") + tk.MustExec("execute stmt2 using @a, @b, @c, @d, @e, @f, @g, @h, @i, @j") + tk.MustQuery("show warnings").Sort().Check(testkit.Rows("Warning 1105 Memory capacity of 1330 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen", + "Warning 1105 skip prepared plan-cache: in-list is too long")) + tk.MustExec("execute stmt2 using @a, @b, @c, @d, @e, @f, @g, @h, @i, @j") + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) +} + +func TestCorColRangeWithRangeMaxSize(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2, t3") + tk.MustExec("create table t1(a int)") + tk.MustExec("create table t2 (a int, b int, c int, index idx_a_b(a, b))") + tk.MustExec("create table t3(a int primary key)") + tk.MustExec("insert into t1 values (2), (4), (6)") + tk.MustExec("insert into t2 (a, b) values (1, 2), (3, 2), (5, 2)") + tk.MustExec("insert into t3 values (2), (4)") + tk.MustExec("insert into mysql.opt_rule_blacklist value(\"decorrelate\")") + tk.MustExec("admin reload opt_rule_blacklist") + defer func() { + tk.MustExec("delete from mysql.opt_rule_blacklist where name = \"decorrelate\"") + tk.MustExec("admin reload opt_rule_blacklist") + }() + + // Correlated column in index range. + tk.MustExec("set @@tidb_opt_range_max_size=1000") + rows := tk.MustQuery("explain format='brief' select * from t1 where exists (select * from t2 where t2.a in (1, 3, 5) and b >= 2 and t2.b = t1.a)").Rows() + // 1000 is not enough for [1 2,1 +inf], [3 2,3 +inf], [5 2,5 +inf]. So b >= 2 is not used to build ranges. + require.True(t, strings.Contains(rows[4][0].(string), "Selection")) + require.True(t, strings.Contains(rows[4][4].(string), "ge(test.t2.b, 2)")) + // 1000 is not enough for [1 ?,1 ?], [3 ?,3 ?], [5 ?,5 ?] but we don't restrict range mem usage when appending col = cor_col + // conditions to access conditions in SplitCorColAccessCondFromFilters. + require.True(t, strings.Contains(rows[5][0].(string), "IndexRangeScan")) + require.True(t, strings.Contains(rows[5][4].(string), "range: decided by [in(test.t2.a, 1, 3, 5) eq(test.t2.b, test.t1.a)]")) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 Memory capacity of 1000 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen")) + // We need to rebuild index ranges each time the value of correlated column test.t1.a changes. We don't restrict range + // mem usage when rebuilding index ranges, otherwise range fallback would happen when rebuilding index ranges, causing + // to wrong query results. + tk.MustQuery("select * from t1 where exists (select * from t2 where t2.a in (1, 3, 5) and b >= 2 and t2.b = t1.a)").Check(testkit.Rows("2")) + + // Correlated column in table range. + tk.MustExec("set @@tidb_opt_range_max_size=1") + rows = tk.MustQuery("explain format='brief' select * from t1 where exists (select * from t3 where t3.a = t1.a)").Rows() + // 1 is not enough for [?,?] but we don't restrict range mem usage when adding col = cor_col to access conditions. + require.True(t, strings.Contains(rows[4][0].(string), "TableRangeScan")) + require.True(t, strings.Contains(rows[4][4].(string), "range: decided by [eq(test.t3.a, test.t1.a)]")) + tk.MustQuery("show warnings").Check(testkit.Rows()) + // We need to rebuild table ranges each time the value of correlated column test.t1.a changes. We don't restrict range + // mem usage when rebuilding table ranges, otherwise range fallback would happen when rebuilding table ranges, causing + // to wrong query results. + tk.MustQuery("select * from t1 where exists (select * from t3 where t3.a = t1.a)").Check(testkit.Rows("2", "4")) +} + +// TestExplainAnalyzeDMLCommit covers the issue #37373. +func TestExplainAnalyzeDMLCommit(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (c1 int key, c2 int);") + tk.MustExec("insert into t values (1, 1)") + + err := failpoint.Enable("github.com/pingcap/tidb/pkg/session/mockSleepBeforeTxnCommit", "return(500)") + require.NoError(t, err) + defer func() { + _ = failpoint.Disable("github.com/pingcap/tidb/pkg/session/mockSleepBeforeTxnCommit") + }() + // The commit is paused by the failpoint, after the fix the explain statement + // execution should proceed after the commit finishes. + _, err = tk.Exec("explain analyze delete from t;") + require.NoError(t, err) + tk.MustQuery("select * from t").Check(testkit.Rows()) +} + +func TestPlanCacheForIndexJoinRangeFallback(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`set @@tidb_enable_prepared_plan_cache=1`) + tk.MustExec("set @@tidb_enable_collect_execution_info=0") + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1(a int, b varchar(10), c varchar(10), index idx_a_b(a, b))") + tk.MustExec("create table t2(d int)") + tk.MustExec("set @@tidb_opt_range_max_size=1260") + // 1260 is enough for [? a,? a], [? b,? b], [? c,? c] but is not enough for [? aaaaaa,? aaaaaa], [? bbbbbb,? bbbbbb], [? cccccc,? cccccc]. + rows := tk.MustQuery("explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.d where t1.b in ('a', 'b', 'c')").Rows() + require.True(t, strings.Contains(rows[6][4].(string), "range: decided by [eq(test.t1.a, test.t2.d) in(test.t1.b, a, b, c)]")) + tk.MustQuery("show warnings").Check(testkit.Rows()) + rows = tk.MustQuery("explain format='brief' select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.d where t1.b in ('aaaaaa', 'bbbbbb', 'cccccc');").Rows() + require.Contains(t, rows[6][4].(string), "range: decided by [eq(test.t1.a, test.t2.d)]") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 Memory capacity of 1260 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen")) + + tk.MustExec("prepare stmt1 from 'select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.d where t1.b in (?, ?, ?)'") + tk.MustExec("set @a='a', @b='b', @c='c'") + tk.MustExec("execute stmt1 using @a, @b, @c") + tk.MustQuery("show warnings").Check(testkit.Rows()) + tk.MustExec("set @a='aaaaaa', @b='bbbbbb', @c='cccccc'") + tk.MustExec("execute stmt1 using @a, @b, @c") + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + tk.MustExec("execute stmt1 using @a, @b, @c") + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) + rows = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() + // We don't limit range mem usage when rebuilding index join ranges for the cached plan. So [? aaaaaa,? aaaaaa], [? bbbbbb,? bbbbbb], [? cccccc,? cccccc] can be built. + require.Contains(t, rows[6][4].(string), "range: decided by [eq(test.t1.a, test.t2.d) in(test.t1.b, aaaaaa, bbbbbb, cccccc)]") + + // Test the plan with range fallback would not be put into cache. + tk.MustExec("prepare stmt2 from 'select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.d where t1.b in (?, ?, ?, ?, ?)'") + tk.MustExec("set @a='a', @b='b', @c='c', @d='d', @e='e'") + tk.MustExec("execute stmt2 using @a, @b, @c, @d, @e") + tk.MustQuery("show warnings").Sort().Check(testkit.Rows("Warning 1105 Memory capacity of 1260 bytes for 'tidb_opt_range_max_size' exceeded when building ranges. Less accurate ranges such as full range are chosen", + "Warning 1105 skip prepared plan-cache: in-list is too long")) + tk.MustExec("execute stmt2 using @a, @b, @c, @d, @e") + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) +} + +func TestIsIPv4ToTiFlash(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(v4 varchar(100), v6 varchar(100))") + tk.MustExec("insert into t values('123.123.123.123', 'F746:C349:48E3:22F2:81E0:0EA8:E7B6:8286')") + tk.MustExec("insert into t values('0.0.0.0', '0000:0000:0000:0000:0000:0000:0000:0000')") + tk.MustExec("insert into t values('127.0.0.1', '2001:0:2851:b9f0:6d:2326:9036:f37a')") + + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + + // Create virtual tiflash replica info. + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + rows := [][]interface{}{ + {"TableReader_10", "root", "MppVersion: 2, data:ExchangeSender_9"}, + {"└─ExchangeSender_9", "mpp[tiflash]", "ExchangeType: PassThrough"}, + {" └─Projection_4", "mpp[tiflash]", "is_ipv4(test.t.v4)->Column#4"}, + {" └─TableFullScan_8", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select is_ipv4(v4) from t;").CheckAt([]int{0, 2, 4}, rows) +} + +func TestIsIPv6ToTiFlash(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(v4 varchar(100), v6 varchar(100))") + tk.MustExec("insert into t values('123.123.123.123', 'F746:C349:48E3:22F2:81E0:0EA8:E7B6:8286')") + tk.MustExec("insert into t values('0.0.0.0', '0000:0000:0000:0000:0000:0000:0000:0000')") + tk.MustExec("insert into t values('127.0.0.1', '2001:0:2851:b9f0:6d:2326:9036:f37a')") + + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + + // Create virtual tiflash replica info. + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + rows := [][]interface{}{ + {"TableReader_10", "root", "MppVersion: 2, data:ExchangeSender_9"}, + {"└─ExchangeSender_9", "mpp[tiflash]", "ExchangeType: PassThrough"}, + {" └─Projection_4", "mpp[tiflash]", "is_ipv6(test.t.v6)->Column#4"}, + {" └─TableFullScan_8", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select is_ipv6(v6) from t;").CheckAt([]int{0, 2, 4}, rows) +} + +// https://github.com/pingcap/tidb/issues/41355 +// The "virtual generated column" push down is not supported now. +// This test covers: TopN, Projection, Selection. +func TestVirtualExprPushDown(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t;") + tk.MustExec("CREATE TABLE t (c1 int DEFAULT 0, c2 int GENERATED ALWAYS AS (abs(c1)) VIRTUAL);") + tk.MustExec("insert into t(c1) values(1), (-1), (2), (-2), (99), (-99);") + tk.MustExec("set @@tidb_isolation_read_engines = 'tikv'") + tk.MustExec("set @@session.tidb_allow_tiflash_cop=ON") + + // TopN to tikv. + rows := [][]interface{}{ + {"TopN_7", "root", "test.t.c2, offset:0, count:2"}, + {"└─TableReader_13", "root", "data:TableFullScan_12"}, + {" └─TableFullScan_12", "cop[tikv]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select * from t order by c2 limit 2;").CheckAt([]int{0, 2, 4}, rows) + + // Projection to tikv. + rows = [][]interface{}{ + {"Projection_3", "root", "plus(test.t.c1, test.t.c2)->Column#4"}, + {"└─TableReader_5", "root", "data:TableFullScan_4"}, + {" └─TableFullScan_4", "cop[tikv]", "keep order:false, stats:pseudo"}, + } + tk.MustExec("set session tidb_opt_projection_push_down='ON';") + tk.MustQuery("explain select c1 + c2 from t;").CheckAt([]int{0, 2, 4}, rows) + tk.MustExec("set session tidb_opt_projection_push_down='OFF';") + + // Selection to tikv. + rows = [][]interface{}{ + {"Selection_7", "root", "gt(test.t.c2, 1)"}, + {"└─TableReader_6", "root", "data:TableFullScan_5"}, + {" └─TableFullScan_5", "cop[tikv]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select * from t where c2 > 1;").CheckAt([]int{0, 2, 4}, rows) + + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + // TopN to tiflash. + rows = [][]interface{}{ + {"TopN_7", "root", "test.t.c2, offset:0, count:2"}, + {"└─TableReader_15", "root", "data:TableFullScan_14"}, + {" └─TableFullScan_14", "cop[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select * from t order by c2 limit 2;").CheckAt([]int{0, 2, 4}, rows) + + // Projection to tiflash. + rows = [][]interface{}{ + {"Projection_3", "root", "plus(test.t.c1, test.t.c2)->Column#4"}, + {"└─TableReader_6", "root", "data:TableFullScan_5"}, + {" └─TableFullScan_5", "cop[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustExec("set session tidb_opt_projection_push_down='ON';") + tk.MustQuery("explain select c1 + c2 from t;").CheckAt([]int{0, 2, 4}, rows) + tk.MustExec("set session tidb_opt_projection_push_down='OFF';") + + // Selection to tiflash. + rows = [][]interface{}{ + {"Selection_8", "root", "gt(test.t.c2, 1)"}, + {"└─TableReader_7", "root", "data:TableFullScan_6"}, + {" └─TableFullScan_6", "cop[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain select * from t where c2 > 1;").CheckAt([]int{0, 2, 4}, rows) +} + +func TestWindowRangeFramePushDownTiflash(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists test.first_range;") + tk.MustExec("create table test.first_range(p int not null, o int not null, v int not null, o_datetime datetime not null, o_time time not null);") + tk.MustExec("insert into test.first_range (p, o, v, o_datetime, o_time) values (0, 0, 0, '2023-9-20 11:17:10', '11:17:10');") + + tk.MustExec("drop table if exists test.first_range_d64;") + tk.MustExec("create table test.first_range_d64(p int not null, o decimal(17,1) not null, v int not null);") + tk.MustExec("insert into test.first_range_d64 (p, o, v) values (0, 0.1, 0), (1, 1.0, 1), (1, 2.1, 2), (1, 4.1, 4), (1, 8.1, 8), (2, 0.0, 0), (2, 3.1, 3), (2, 10.0, 10), (2, 13.1, 13), (2, 15.1, 15), (3, 1.1, 1), (3, 2.9, 3), (3, 5.1, 5), (3, 9.1, 9), (3, 15.0, 15), (3, 20.1, 20), (3, 31.1, 31);") + + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + + // Create virtual tiflash replica info. + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "first_range" || tblInfo.Name.L == "first_range_d64" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec(`set @@tidb_max_tiflash_threads=20`) + + tk.MustQuery("explain select *, first_value(v) over (partition by p order by o range between 3 preceding and 0 following) as a from test.first_range;").Check(testkit.Rows( + "TableReader_23 10000.00 root MppVersion: 2, data:ExchangeSender_22", + "└─ExchangeSender_22 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Window_21 10000.00 mpp[tiflash] first_value(test.first_range.v)->Column#8 over(partition by test.first_range.p order by test.first_range.o range between 3 preceding and 0 following), stream_count: 20", + " └─Sort_13 10000.00 mpp[tiflash] test.first_range.p, test.first_range.o, stream_count: 20", + " └─ExchangeReceiver_12 10000.00 mpp[tiflash] stream_count: 20", + " └─ExchangeSender_11 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.first_range.p, collate: binary], stream_count: 20", + " └─TableFullScan_10 10000.00 mpp[tiflash] table:first_range keep order:false, stats:pseudo")) + + tk.MustQuery("explain select *, first_value(v) over (partition by p order by o range between 3 preceding and 2.9E0 following) as a from test.first_range;").Check(testkit.Rows( + "TableReader_23 10000.00 root MppVersion: 2, data:ExchangeSender_22", + "└─ExchangeSender_22 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Window_21 10000.00 mpp[tiflash] first_value(test.first_range.v)->Column#8 over(partition by test.first_range.p order by test.first_range.o range between 3 preceding and 2.9 following), stream_count: 20", + " └─Sort_13 10000.00 mpp[tiflash] test.first_range.p, test.first_range.o, stream_count: 20", + " └─ExchangeReceiver_12 10000.00 mpp[tiflash] stream_count: 20", + " └─ExchangeSender_11 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.first_range.p, collate: binary], stream_count: 20", + " └─TableFullScan_10 10000.00 mpp[tiflash] table:first_range keep order:false, stats:pseudo")) + + tk.MustQuery("explain select *, first_value(v) over (partition by p order by o range between 2.3 preceding and 0 following) as a from test.first_range_d64;").Check(testkit.Rows( + "TableReader_23 10000.00 root MppVersion: 2, data:ExchangeSender_22", + "└─ExchangeSender_22 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Window_21 10000.00 mpp[tiflash] first_value(test.first_range_d64.v)->Column#6 over(partition by test.first_range_d64.p order by test.first_range_d64.o range between 2.3 preceding and 0 following), stream_count: 20", + " └─Sort_13 10000.00 mpp[tiflash] test.first_range_d64.p, test.first_range_d64.o, stream_count: 20", + " └─ExchangeReceiver_12 10000.00 mpp[tiflash] stream_count: 20", + " └─ExchangeSender_11 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.first_range_d64.p, collate: binary], stream_count: 20", + " └─TableFullScan_10 10000.00 mpp[tiflash] table:first_range_d64 keep order:false, stats:pseudo")) + + tk.MustQuery("explain select *, first_value(v) over (partition by p order by o_datetime range between interval 1 day preceding and interval 1 day following) as a from test.first_range;").Check(testkit.Rows( + "TableReader_23 10000.00 root MppVersion: 2, data:ExchangeSender_22", + "└─ExchangeSender_22 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Window_21 10000.00 mpp[tiflash] first_value(test.first_range.v)->Column#8 over(partition by test.first_range.p order by test.first_range.o_datetime range between interval 1 \"DAY\" preceding and interval 1 \"DAY\" following), stream_count: 20", + " └─Sort_13 10000.00 mpp[tiflash] test.first_range.p, test.first_range.o_datetime, stream_count: 20", + " └─ExchangeReceiver_12 10000.00 mpp[tiflash] stream_count: 20", + " └─ExchangeSender_11 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.first_range.p, collate: binary], stream_count: 20", + " └─TableFullScan_10 10000.00 mpp[tiflash] table:first_range keep order:false, stats:pseudo")) + + tk.MustQuery("explain select *, first_value(v) over (partition by p order by o_time range between interval 1 day preceding and interval 1 day following) as a from test.first_range;").Check(testkit.Rows( + "Shuffle_13 10000.00 root execution info: concurrency:5, data sources:[TableReader_11]", + "└─Window_8 10000.00 root first_value(test.first_range.v)->Column#8 over(partition by test.first_range.p order by test.first_range.o_time range between interval 1 \"DAY\" preceding and interval 1 \"DAY\" following)", + " └─Sort_12 10000.00 root test.first_range.p, test.first_range.o_time", + " └─TableReader_11 10000.00 root MppVersion: 2, data:ExchangeSender_10", + " └─ExchangeSender_10 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan_9 10000.00 mpp[tiflash] table:first_range keep order:false, stats:pseudo")) +} + +// https://github.com/pingcap/tidb/issues/41458 +func TestIssue41458(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil, nil)) + tk.MustExec("use test") + tk.MustExec(`create table t (a int, b int, c int, index ia(a));`) + tk.MustExec("select * from t t1 join t t2 on t1.b = t2.b join t t3 on t2.b=t3.b join t t4 on t3.b=t4.b where t3.a=1 and t2.a=2;") + rawRows := tk.MustQuery("select plan from information_schema.statements_summary where SCHEMA_NAME = 'test' and STMT_TYPE = 'Select';").Sort().Rows() + plan := rawRows[0][0].(string) + rows := strings.Split(plan, "\n") + rows = rows[1:] + expectedRes := []string{ + "Projection", + "└─HashJoin", + " ├─HashJoin", + " │ ├─HashJoin", + " │ │ ├─IndexLookUp", + " │ │ │ ├─IndexRangeScan", + " │ │ │ └─Selection", + " │ │ │ └─TableRowIDScan", + " │ │ └─IndexLookUp", + " │ │ ├─IndexRangeScan", + " │ │ └─Selection", + " │ │ └─TableRowIDScan", + " │ └─TableReader", + " │ └─Selection", + " │ └─TableFullScan", + " └─TableReader", + " └─Selection", + " └─TableFullScan", + } + for i, row := range rows { + fields := strings.Split(row, "\t") + fields = strings.Split(fields[1], "_") + op := fields[0] + require.Equalf(t, expectedRes[i], op, fmt.Sprintf("Mismatch at index %d.", i)) + } +} diff --git a/pkg/planner/core/testdata/index_merge_suite_out.json b/pkg/planner/core/testdata/index_merge_suite_out.json new file mode 100644 index 0000000000000..cfa13020f7c1a --- /dev/null +++ b/pkg/planner/core/testdata/index_merge_suite_out.json @@ -0,0 +1,436 @@ +[ + { + "Name": "TestIndexMergePathGeneration", + "Cases": [ + "[]", + "[]", + "[{Idxs:[c_d_e,f],TbFilters:[]}]", + "[{Idxs:[c_d_e,f],TbFilters:[or(gt(test.t.c, 5), lt(test.t.f, 7))]},{Idxs:[c_d_e,f],TbFilters:[or(lt(test.t.c, 1), gt(test.t.f, 2))]}]", + "[{Idxs:[c_d_e,f],TbFilters:[or(gt(test.t.c, 5), lt(test.t.f, 7)),or(lt(test.t.c, 1), gt(test.t.g, 2))]},{Idxs:[c_d_e,f],TbFilters:[or(lt(test.t.c, 1), gt(test.t.f, 2)),or(lt(test.t.c, 1), gt(test.t.g, 2))]},{Idxs:[c_d_e,g],TbFilters:[or(lt(test.t.c, 1), gt(test.t.f, 2)),or(gt(test.t.c, 5), lt(test.t.f, 7))]}]", + "[{Idxs:[c_d_e,f],TbFilters:[or(gt(test.t.c, 5), lt(test.t.f, 7)),or(lt(test.t.e, 1), gt(test.t.f, 2))]},{Idxs:[c_d_e,f],TbFilters:[or(lt(test.t.c, 1), gt(test.t.f, 2)),or(lt(test.t.e, 1), gt(test.t.f, 2))]}]" + ] + }, + { + "Name": "TestHintForIntersectionIndexMerge", + "Cases": [ + { + "SQL": "set @@tidb_partition_prune_mode = 'dynamic'", + "Plan": null, + "Result": null + }, + { + "SQL": "select * from vh", + "Plan": [ + "IndexMerge 0.97 root partition:p0,p1,p2 type: intersection", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:ia(a) range:[10,10], keep order:false", + "├─IndexRangeScan(Build) 2.60 cop[tikv] table:t1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t1, index:id(d) range:[2,2], [5,5], keep order:false", + "└─TableRowIDScan(Probe) 0.97 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "Plan": [ + "IndexMerge 0.97 root partition:p0,p1,p2 type: intersection", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:ia(a) range:[10,10], keep order:false", + "├─IndexRangeScan(Build) 2.60 cop[tikv] table:t1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t1, index:id(d) range:[2,2], [5,5], keep order:false", + "└─TableRowIDScan(Probe) 0.97 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "Plan": [ + "IndexMerge 0.97 root partition:p0,p1,p2 type: intersection", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:ia(a) range:[10,10], keep order:false", + "├─IndexRangeScan(Build) 2.60 cop[tikv] table:t1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t1, index:id(d) range:[2,2], [5,5], keep order:false", + "└─TableRowIDScan(Probe) 0.97 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "Plan": [ + "IndexMerge 0.97 root partition:p0,p1,p2 type: intersection", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:ia(a) range:[10,10], keep order:false", + "├─IndexRangeScan(Build) 2.60 cop[tikv] table:t1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t1, index:id(d) range:[2,2], [5,5], keep order:false", + "└─TableRowIDScan(Probe) 0.97 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v1@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v1 where c < 30 and d in (2,5)", + "Plan": [ + "IndexMerge 0.97 root partition:p0,p1,p2 type: intersection", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, index:ia(a) range:[10,10], keep order:false", + "├─IndexRangeScan(Build) 2.60 cop[tikv] table:t1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t1, index:id(d) range:[2,2], [5,5], keep order:false", + "└─TableRowIDScan(Probe) 0.97 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ use_index_merge(t2, ia, ibc, id, ie) */ * from t2 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", + "Plan": [ + "IndexMerge 0.89 root partition:p0,p1,p2 type: intersection", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, index:ia(a) range:(10,+inf], keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t2, index:id(d) range:[-inf,45), keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, index:ie(e) range:[100,100], keep order:false", + "└─TableRowIDScan(Probe) 0.89 cop[tikv] table:t2 keep order:false" + ], + "Result": [ + "20 20 20 5 100" + ] + }, + { + "SQL": "select /*+ use_index_merge(t3, ia, ibc, id, ie) */ * from t3 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", + "Plan": [ + "IndexMerge 0.44 root partition:p0 type: intersection", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t3, index:ia(a) range:(10,+inf], keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t3, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t3, index:id(d) range:[-inf,45), keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t3, index:ie(e) range:[100,100], keep order:false", + "└─TableRowIDScan(Probe) 0.44 cop[tikv] table:t3 keep order:false" + ], + "Result": [ + "20 20 20 5 100" + ] + }, + { + "SQL": "select /*+ use_index_merge(t4, ia, ibc, id, ie) */ * from t4 where a > 10 and b = 20 and c < 35 and d in (1,3,8,9) and e = 100", + "Plan": [ + "IndexMerge 0.89 root partition:p0,p1 type: intersection", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t4, index:ia(a) range:(10,+inf], keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t4, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + "├─IndexRangeScan(Build) 3.00 cop[tikv] table:t4, index:id(d) range:[1,1], [3,3], [8,8], [9,9], keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t4, index:ie(e) range:[100,100], keep order:false", + "└─TableRowIDScan(Probe) 0.89 cop[tikv] table:t4 keep order:false" + ], + "Result": [ + "30 20 5 8 100" + ] + }, + { + "SQL": "select /*+ use_index_merge(t5, is1, is2, is3, is4) */ * from t5 where s1 = 'Abc' and s2 > 'zzz' and s3 < 'B啊a' and s4 = 'CcC'", + "Plan": [ + "IndexMerge 0.00 root type: intersection", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t5, index:is1(s1) range:[\"Abc\",\"Abc\"], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t5, index:is2(s2) range:(\"zzz\",+inf], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t5, index:is3(s3) range:[-inf,\"\\x0eJ\\xfb@\\xd5J\\x0e3\"), keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t5, index:is4(s4) range:[\"CCC\",\"CCC\"], keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 0.00 cop[tikv] table:t5 keep order:false, stats:pseudo" + ], + "Result": [ + "Abc zzzz aa ccc" + ] + }, + { + "SQL": "select /*+ use_index_merge(t6, primary, is3, is4) */ * from t6 where s1 = 'Abc' and s2 > 'zzz' and s3 = 'A啊a' and s4 not like 'Cd_'", + "Plan": [ + "IndexMerge 0.03 root type: intersection", + "├─IndexRangeScan(Build) 33.33 cop[tikv] table:t6, index:PRIMARY(s1, s2) range:(\"Abc\" \"zzz\",\"Abc\" +inf], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t6, index:is3(s3) range:[\"\\x0e3\\xfb@\\xd5J\\x0e3\",\"\\x0e3\\xfb@\\xd5J\\x0e3\"], keep order:false, stats:pseudo", + "└─Selection(Probe) 0.03 cop[tikv] gt(test.t6.s2, \"zzz\"), not(like(test.t6.s4, \"Cd_\", 92))", + " └─TableRowIDScan 0.03 cop[tikv] table:t6 keep order:false, stats:pseudo" + ], + "Result": [ + "Abc zzzz A啊A Cdaa" + ] + }, + { + "SQL": "select /*+ use_index_merge(t7, primary,ia,ib,ic,ie,iff,ig) */ * from t7 where a = 100 and b > 5 and c < 12.3 and d > 54.321 and e = '2022-11-22 17:00' and f > '2020-6-23 10:00' and g < 2025", + "Plan": [ + "IndexMerge 0.00 root type: intersection", + "├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t7, index:PRIMARY(d) range:(54.321,+inf], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t7, index:ia(a) range:[100,100], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t7, index:ib(b) range:(\"0x05\",+inf], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t7, index:ic(c) range:[-inf,12.3), keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t7, index:ie(e) range:[2022-11-22 17:00:00,2022-11-22 17:00:00], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t7, index:iff(f) range:(2020-06-23 10:00:00.00000,+inf], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t7, index:ig(g) range:[-inf,2025), keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 0.00 cop[tikv] table:t7 keep order:false, stats:pseudo" + ], + "Result": [ + "100 \u0006 12.2 56.000 2022-11-22 17:00:00 2022-12-21 00:00:00.00000 2021" + ] + }, + { + "SQL": "select /*+ use_index_merge(t8, primary,is2,is3,is4,is5) */ * from t8 where s1 like '啊A%' and s2 > 'abc' and s3 > 'cba' and s4 in ('aA', '??') and s5 = 'test,2'", + "Plan": [ + "Selection 0.04 root eq(test.t8.s5, \"test,2\")", + "└─IndexMerge 0.06 root type: intersection", + " ├─IndexRangeScan(Build) 250.00 cop[tikv] table:t8, index:PRIMARY(s1) range:[\"UJ\\x00A\",\"UJ\\x00B\"), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t8, index:is2(s2) range:(\"abc\",+inf], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t8, index:is3(s3) range:(\"cba\",+inf], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t8, index:is4(s4) range:[\"aA\",\"aA\"], [\"??\",\"??\"], keep order:false, stats:pseudo", + " └─Selection(Probe) 0.06 cop[tikv] gt(test.t8.s3, \"cba\"), like(test.t8.s1, \"啊A%\", 92)", + " └─TableRowIDScan 0.06 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Result": [ + "啊aabbccdd abcc cccc aA tEsT,2" + ] + }, + { + "SQL": "select (select /*+ use_index_merge(t1,ia,ibc,ic) */ a from t1 where t1.a > 10 and t1.b = 20 and t1.c = t2.a) from t2", + "Plan": [ + "Projection 3.00 root test.t1.a", + "└─Apply 3.00 root CARTESIAN left outer join", + " ├─IndexReader(Build) 3.00 root partition:all index:IndexFullScan", + " │ └─IndexFullScan 3.00 cop[tikv] table:t2, index:ia(a) keep order:false", + " └─MaxOneRow(Probe) 3.00 root ", + " └─IndexMerge 1.50 root partition:all type: intersection", + " ├─IndexRangeScan(Build) 6.00 cop[tikv] table:t1, index:ia(a) range:(10,+inf], keep order:false", + " ├─IndexRangeScan(Build) 4.00 cop[tikv] table:t1, index:ibc(b, c) range: decided by [eq(test.t1.b, 20) eq(test.t1.c, test.t2.a)], keep order:false", + " └─TableRowIDScan(Probe) 1.50 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "20", + "", + "" + ] + }, + { + "SQL": "select (select /*+ use_index_merge(t1,ia,ibc,ic) */ a from t1 where t1.a > 10 and t1.b = 20 and t1.c > t2.a) from t2", + "Plan": [ + "Projection 3.00 root test.t1.a", + "└─Apply 3.00 root CARTESIAN left outer join", + " ├─IndexReader(Build) 3.00 root partition:all index:IndexFullScan", + " │ └─IndexFullScan 3.00 cop[tikv] table:t2, index:ia(a) keep order:false", + " └─MaxOneRow(Probe) 3.00 root ", + " └─IndexMerge 3.60 root partition:all type: intersection", + " ├─IndexRangeScan(Build) 6.00 cop[tikv] table:t1, index:ia(a) range:(10,+inf], keep order:false", + " ├─Selection(Build) 7.20 cop[tikv] gt(test.t1.c, test.t2.a)", + " │ └─IndexRangeScan 9.00 cop[tikv] table:t1, index:ibc(b, c) range:[20,20], keep order:false", + " └─TableRowIDScan(Probe) 3.60 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "20", + "20", + "" + ] + }, + { + "SQL": "select (select /*+ use_index_merge(t1,ia,ibc,ic) */ a from t1 where t1.a > 10 and t1.b = 20 and t1.e > t2.a) from t2", + "Plan": [ + "Projection 3.00 root test.t1.a", + "└─Apply 3.00 root CARTESIAN left outer join", + " ├─IndexReader(Build) 3.00 root partition:all index:IndexFullScan", + " │ └─IndexFullScan 3.00 cop[tikv] table:t2, index:ia(a) keep order:false", + " └─MaxOneRow(Probe) 3.00 root ", + " └─IndexMerge 3.60 root partition:all type: intersection", + " ├─IndexRangeScan(Build) 6.00 cop[tikv] table:t1, index:ia(a) range:(10,+inf], keep order:false", + " ├─IndexRangeScan(Build) 9.00 cop[tikv] table:t1, index:ibc(b, c) range:[20,20], keep order:false", + " └─Selection(Probe) 3.60 cop[tikv] gt(test.t1.e, test.t2.a)", + " └─TableRowIDScan 4.50 cop[tikv] table:t1 keep order:false" + ], + "Result": [ + "20", + "20", + "20" + ] + }, + { + "SQL": "set @@tidb_partition_prune_mode = 'static'", + "Plan": null, + "Result": null + }, + { + "SQL": "select * from vh", + "Plan": [ + "PartitionUnion 0.50 root ", + "├─IndexMerge 0.50 root type: intersection", + "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 0.50 cop[tikv] table:t1, partition:p0 keep order:false", + "├─IndexMerge 1.00 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p1 keep order:false", + "└─IndexMerge 0.00 root type: intersection", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, partition:p2, index:ia(a) range:[10,10], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t1, partition:p2, index:ibc(b, c) range:[20 -inf,20 30), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t1, partition:p2, index:id(d) range:[2,2], [5,5], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "Plan": [ + "PartitionUnion 0.50 root ", + "├─IndexMerge 0.50 root type: intersection", + "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 0.50 cop[tikv] table:t1, partition:p0 keep order:false", + "├─IndexMerge 1.00 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p1 keep order:false", + "└─IndexMerge 0.00 root type: intersection", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, partition:p2, index:ia(a) range:[10,10], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t1, partition:p2, index:ibc(b, c) range:[20 -inf,20 30), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t1, partition:p2, index:id(d) range:[2,2], [5,5], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "Plan": [ + "PartitionUnion 0.50 root ", + "├─IndexMerge 0.50 root type: intersection", + "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 0.50 cop[tikv] table:t1, partition:p0 keep order:false", + "├─IndexMerge 1.00 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p1 keep order:false", + "└─IndexMerge 0.00 root type: intersection", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, partition:p2, index:ia(a) range:[10,10], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t1, partition:p2, index:ibc(b, c) range:[20 -inf,20 30), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t1, partition:p2, index:id(d) range:[2,2], [5,5], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "Plan": [ + "PartitionUnion 0.50 root ", + "├─IndexMerge 0.50 root type: intersection", + "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 0.50 cop[tikv] table:t1, partition:p0 keep order:false", + "├─IndexMerge 1.00 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p1 keep order:false", + "└─IndexMerge 0.00 root type: intersection", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, partition:p2, index:ia(a) range:[10,10], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t1, partition:p2, index:ibc(b, c) range:[20 -inf,20 30), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t1, partition:p2, index:id(d) range:[2,2], [5,5], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ qb_name(v, v@sel_1 .@sel_1), use_index_merge(@v t1, ia, ibc, id) */ * from v", + "Plan": [ + "PartitionUnion 0.50 root ", + "├─IndexMerge 0.50 root type: intersection", + "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t1, partition:p0, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p0, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 0.50 cop[tikv] table:t1, partition:p0 keep order:false", + "├─IndexMerge 1.00 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ia(a) range:[10,10], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:ibc(b, c) range:[20 -inf,20 30), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t1, partition:p1, index:id(d) range:[2,2], [5,5], keep order:false", + "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t1, partition:p1 keep order:false", + "└─IndexMerge 0.00 root type: intersection", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, partition:p2, index:ia(a) range:[10,10], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t1, partition:p2, index:ibc(b, c) range:[20 -inf,20 30), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t1, partition:p2, index:id(d) range:[2,2], [5,5], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t1, partition:p2 keep order:false, stats:pseudo" + ], + "Result": [ + "10 20 5 5 3" + ] + }, + { + "SQL": "select /*+ use_index_merge(t2, ia, ibc, id, ie) */ * from t2 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", + "Plan": [ + "PartitionUnion 1.25 root ", + "├─IndexMerge 0.25 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p0, index:ia(a) range:(10,+inf], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p0, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t2, partition:p0, index:id(d) range:[-inf,45), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p0, index:ie(e) range:[100,100], keep order:false", + "│ └─TableRowIDScan(Probe) 0.25 cop[tikv] table:t2, partition:p0 keep order:false", + "├─IndexMerge 1.00 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p1, index:ia(a) range:(10,+inf], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p1, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p1, index:id(d) range:[-inf,45), keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t2, partition:p1, index:ie(e) range:[100,100], keep order:false", + "│ └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t2, partition:p1 keep order:false", + "└─IndexMerge 0.00 root type: intersection", + " ├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t2, partition:p2, index:ia(a) range:(10,+inf], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 33.23 cop[tikv] table:t2, partition:p2, index:ibc(b, c) range:[20 -inf,20 35), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t2, partition:p2, index:id(d) range:[-inf,45), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t2, partition:p2, index:ie(e) range:[100,100], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t2, partition:p2 keep order:false, stats:pseudo" + ], + "Result": [ + "20 20 20 5 100" + ] + }, + { + "SQL": "select /*+ use_index_merge(t3, ia, ibc, id, ie) */ * from t3 where a > 10 and b = 20 and c < 35 and d < 45 and e = 100", + "Plan": [ + "IndexMerge 0.50 root type: intersection", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t3, partition:p0, index:ia(a) range:(10,+inf], keep order:false", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t3, partition:p0, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t3, partition:p0, index:id(d) range:[-inf,45), keep order:false", + "├─IndexRangeScan(Build) 2.00 cop[tikv] table:t3, partition:p0, index:ie(e) range:[100,100], keep order:false", + "└─TableRowIDScan(Probe) 0.50 cop[tikv] table:t3, partition:p0 keep order:false" + ], + "Result": [ + "20 20 20 5 100" + ] + }, + { + "SQL": "select /*+ use_index_merge(t4, ia, ibc, id, ie) */ * from t4 where a > 10 and b = 20 and c < 35 and d in (1,3,8,9) and e = 100", + "Plan": [ + "PartitionUnion 1.25 root ", + "├─IndexMerge 0.25 root type: intersection", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p0, index:ia(a) range:(10,+inf], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p0, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + "│ ├─IndexRangeScan(Build) 2.00 cop[tikv] table:t4, partition:p0, index:id(d) range:[1,1], [3,3], [8,8], [9,9], keep order:false", + "│ ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p0, index:ie(e) range:[100,100], keep order:false", + "│ └─TableRowIDScan(Probe) 0.25 cop[tikv] table:t4, partition:p0 keep order:false", + "└─IndexMerge 1.00 root type: intersection", + " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p1, index:ia(a) range:(10,+inf], keep order:false", + " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p1, index:ibc(b, c) range:[20 -inf,20 35), keep order:false", + " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p1, index:id(d) range:[1,1], [3,3], [8,8], [9,9], keep order:false", + " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t4, partition:p1, index:ie(e) range:[100,100], keep order:false", + " └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t4, partition:p1 keep order:false" + ], + "Result": [ + "30 20 5 8 100" + ] + } + ] + } +] diff --git a/pkg/util/ranger/BUILD.bazel b/pkg/util/ranger/BUILD.bazel new file mode 100644 index 0000000000000..fa230a789bbf5 --- /dev/null +++ b/pkg/util/ranger/BUILD.bazel @@ -0,0 +1,67 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "ranger", + srcs = [ + "checker.go", + "detacher.go", + "points.go", + "ranger.go", + "types.go", + ], + importpath = "github.com/pingcap/tidb/pkg/util/ranger", + visibility = ["//visibility:public"], + deps = [ + "//pkg/errno", + "//pkg/expression", + "//pkg/kv", + "//pkg/parser/ast", + "//pkg/parser/charset", + "//pkg/parser/format", + "//pkg/parser/model", + "//pkg/parser/mysql", + "//pkg/parser/terror", + "//pkg/planner/util/fixcontrol", + "//pkg/sessionctx", + "//pkg/sessionctx/stmtctx", + "//pkg/types", + "//pkg/types/parser_driver", + "//pkg/util/chunk", + "//pkg/util/codec", + "//pkg/util/collate", + "//pkg/util/dbterror", + "//pkg/util/hack", + "@com_github_pingcap_errors//:errors", + ], +) + +go_test( + name = "ranger_test", + timeout = "short", + srcs = [ + "bench_test.go", + "main_test.go", + "ranger_test.go", + "types_test.go", + ], + flaky = True, + shard_count = 26, + deps = [ + ":ranger", + "//pkg/config", + "//pkg/domain", + "//pkg/expression", + "//pkg/parser/ast", + "//pkg/parser/model", + "//pkg/parser/mysql", + "//pkg/planner/core", + "//pkg/session", + "//pkg/sessionctx", + "//pkg/testkit", + "//pkg/testkit/testsetup", + "//pkg/types", + "//pkg/util/collate", + "@com_github_stretchr_testify//require", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/tests/integrationtest/r/expression/charset_and_collation.result b/tests/integrationtest/r/expression/charset_and_collation.result new file mode 100644 index 0000000000000..4e792011683ba --- /dev/null +++ b/tests/integrationtest/r/expression/charset_and_collation.result @@ -0,0 +1,1993 @@ +drop table if exists t; +create table t (utf8_bin_c varchar(10) charset utf8 collate utf8_bin, utf8_gen_c varchar(10) charset utf8 collate utf8_general_ci, bin_c binary, num_c int, abin char collate ascii_bin, lbin char collate latin1_bin, u4bin char collate utf8mb4_bin, u4ci char collate utf8mb4_general_ci); +insert into t values ('a', 'b', 'c', 4, 'a', 'a', 'a', 'a'); +select collation(null), charset(null); +collation(null) charset(null) +binary binary +select collation(2), charset(2); +collation(2) charset(2) +binary binary +select collation(2 + 'a'), charset(2 + 'a'); +collation(2 + 'a') charset(2 + 'a') +binary binary +select collation(2 + utf8_gen_c), charset(2 + utf8_gen_c) from t; +collation(2 + utf8_gen_c) charset(2 + utf8_gen_c) +binary binary +select collation(2 + utf8_bin_c), charset(2 + utf8_bin_c) from t; +collation(2 + utf8_bin_c) charset(2 + utf8_bin_c) +binary binary +select collation(concat(utf8_bin_c, 2)), charset(concat(utf8_bin_c, 2)) from t; +collation(concat(utf8_bin_c, 2)) charset(concat(utf8_bin_c, 2)) +utf8_bin utf8 +select collation(concat(utf8_gen_c, 'abc')), charset(concat(utf8_gen_c, 'abc')) from t; +collation(concat(utf8_gen_c, 'abc')) charset(concat(utf8_gen_c, 'abc')) +utf8_general_ci utf8 +select collation(concat(utf8_gen_c, null)), charset(concat(utf8_gen_c, null)) from t; +collation(concat(utf8_gen_c, null)) charset(concat(utf8_gen_c, null)) +utf8_general_ci utf8 +select collation(concat(utf8_gen_c, num_c)), charset(concat(utf8_gen_c, num_c)) from t; +collation(concat(utf8_gen_c, num_c)) charset(concat(utf8_gen_c, num_c)) +utf8_general_ci utf8 +select collation(concat(utf8_bin_c, utf8_gen_c)), charset(concat(utf8_bin_c, utf8_gen_c)) from t; +collation(concat(utf8_bin_c, utf8_gen_c)) charset(concat(utf8_bin_c, utf8_gen_c)) +utf8_bin utf8 +select collation(upper(utf8_bin_c)), charset(upper(utf8_bin_c)) from t; +collation(upper(utf8_bin_c)) charset(upper(utf8_bin_c)) +utf8_bin utf8 +select collation(upper(utf8_gen_c)), charset(upper(utf8_gen_c)) from t; +collation(upper(utf8_gen_c)) charset(upper(utf8_gen_c)) +utf8_general_ci utf8 +select collation(upper(bin_c)), charset(upper(bin_c)) from t; +collation(upper(bin_c)) charset(upper(bin_c)) +binary binary +select collation(concat(abin, bin_c)), charset(concat(abin, bin_c)) from t; +collation(concat(abin, bin_c)) charset(concat(abin, bin_c)) +binary binary +select collation(concat(lbin, bin_c)), charset(concat(lbin, bin_c)) from t; +collation(concat(lbin, bin_c)) charset(concat(lbin, bin_c)) +binary binary +select collation(concat(utf8_bin_c, bin_c)), charset(concat(utf8_bin_c, bin_c)) from t; +collation(concat(utf8_bin_c, bin_c)) charset(concat(utf8_bin_c, bin_c)) +binary binary +select collation(concat(utf8_gen_c, bin_c)), charset(concat(utf8_gen_c, bin_c)) from t; +collation(concat(utf8_gen_c, bin_c)) charset(concat(utf8_gen_c, bin_c)) +binary binary +select collation(concat(u4bin, bin_c)), charset(concat(u4bin, bin_c)) from t; +collation(concat(u4bin, bin_c)) charset(concat(u4bin, bin_c)) +binary binary +select collation(concat(u4ci, bin_c)), charset(concat(u4ci, bin_c)) from t; +collation(concat(u4ci, bin_c)) charset(concat(u4ci, bin_c)) +binary binary +select collation(concat(abin, u4bin)), charset(concat(abin, u4bin)) from t; +collation(concat(abin, u4bin)) charset(concat(abin, u4bin)) +utf8mb4_bin utf8mb4 +select collation(concat(lbin, u4bin)), charset(concat(lbin, u4bin)) from t; +collation(concat(lbin, u4bin)) charset(concat(lbin, u4bin)) +utf8mb4_bin utf8mb4 +select collation(concat(utf8_bin_c, u4bin)), charset(concat(utf8_bin_c, u4bin)) from t; +collation(concat(utf8_bin_c, u4bin)) charset(concat(utf8_bin_c, u4bin)) +utf8mb4_bin utf8mb4 +select collation(concat(utf8_gen_c, u4bin)), charset(concat(utf8_gen_c, u4bin)) from t; +collation(concat(utf8_gen_c, u4bin)) charset(concat(utf8_gen_c, u4bin)) +utf8mb4_bin utf8mb4 +select collation(concat(u4ci, u4bin)), charset(concat(u4ci, u4bin)) from t; +collation(concat(u4ci, u4bin)) charset(concat(u4ci, u4bin)) +utf8mb4_bin utf8mb4 +select collation(concat(abin, u4ci)), charset(concat(abin, u4ci)) from t; +collation(concat(abin, u4ci)) charset(concat(abin, u4ci)) +utf8mb4_general_ci utf8mb4 +select collation(concat(lbin, u4ci)), charset(concat(lbin, u4ci)) from t; +collation(concat(lbin, u4ci)) charset(concat(lbin, u4ci)) +utf8mb4_general_ci utf8mb4 +select collation(concat(utf8_bin_c, u4ci)), charset(concat(utf8_bin_c, u4ci)) from t; +collation(concat(utf8_bin_c, u4ci)) charset(concat(utf8_bin_c, u4ci)) +utf8mb4_general_ci utf8mb4 +select collation(concat(utf8_gen_c, u4ci)), charset(concat(utf8_gen_c, u4ci)) from t; +collation(concat(utf8_gen_c, u4ci)) charset(concat(utf8_gen_c, u4ci)) +utf8mb4_general_ci utf8mb4 +select collation(concat(abin, utf8_bin_c)), charset(concat(abin, utf8_bin_c)) from t; +collation(concat(abin, utf8_bin_c)) charset(concat(abin, utf8_bin_c)) +utf8_bin utf8 +select collation(concat(lbin, utf8_bin_c)), charset(concat(lbin, utf8_bin_c)) from t; +collation(concat(lbin, utf8_bin_c)) charset(concat(lbin, utf8_bin_c)) +utf8_bin utf8 +select collation(concat(utf8_gen_c, utf8_bin_c)), charset(concat(utf8_gen_c, utf8_bin_c)) from t; +collation(concat(utf8_gen_c, utf8_bin_c)) charset(concat(utf8_gen_c, utf8_bin_c)) +utf8_bin utf8 +select collation(concat(abin, utf8_gen_c)), charset(concat(abin, utf8_gen_c)) from t; +collation(concat(abin, utf8_gen_c)) charset(concat(abin, utf8_gen_c)) +utf8_general_ci utf8 +select collation(concat(lbin, utf8_gen_c)), charset(concat(lbin, utf8_gen_c)) from t; +collation(concat(lbin, utf8_gen_c)) charset(concat(lbin, utf8_gen_c)) +utf8_general_ci utf8 +select collation(concat(abin, lbin)), charset(concat(abin, lbin)) from t; +collation(concat(abin, lbin)) charset(concat(abin, lbin)) +latin1_bin latin1 +set names utf8mb4 collate utf8mb4_bin; +select collation('a'), charset('a'); +collation('a') charset('a') +utf8mb4_bin utf8mb4 +set names utf8mb4 collate utf8mb4_general_ci; +select collation('a'), charset('a'); +collation('a') charset('a') +utf8mb4_general_ci utf8mb4 +set names utf8mb4 collate utf8mb4_general_ci; +set @test_collate_var = 'a'; +select collation(@test_collate_var), charset(@test_collate_var); +collation(@test_collate_var) charset(@test_collate_var) +utf8mb4_general_ci utf8mb4 +set @test_collate_var = concat("a", "b" collate utf8mb4_bin); +select collation(@test_collate_var), charset(@test_collate_var); +collation(@test_collate_var) charset(@test_collate_var) +utf8mb4_bin utf8mb4 +select locate('1', '123' collate utf8mb4_bin, 2 collate `binary`); +locate('1', '123' collate utf8mb4_bin, 2 collate `binary`) +0 +select 1 in ('a' collate utf8mb4_bin, 'b' collate utf8mb4_general_ci); +1 in ('a' collate utf8mb4_bin, 'b' collate utf8mb4_general_ci) +0 +select left('abc' collate utf8mb4_bin, 2 collate `binary`); +left('abc' collate utf8mb4_bin, 2 collate `binary`) +ab +select right('abc' collate utf8mb4_bin, 2 collate `binary`); +right('abc' collate utf8mb4_bin, 2 collate `binary`) +bc +select repeat('abc' collate utf8mb4_bin, 2 collate `binary`); +repeat('abc' collate utf8mb4_bin, 2 collate `binary`) +abcabc +select trim(both 'abc' collate utf8mb4_bin from 'c' collate utf8mb4_general_ci); +trim(both 'abc' collate utf8mb4_bin from 'c' collate utf8mb4_general_ci) +c +select substr('abc' collate utf8mb4_bin, 2 collate `binary`); +substr('abc' collate utf8mb4_bin, 2 collate `binary`) +bc +select replace('abc' collate utf8mb4_bin, 'b' collate utf8mb4_general_ci, 'd' collate utf8mb4_unicode_ci); +replace('abc' collate utf8mb4_bin, 'b' collate utf8mb4_general_ci, 'd' collate utf8mb4_unicode_ci) +adc +set names default; +drop table if exists t; +create table t (a char(10) charset gbk collate gbk_chinese_ci, b time); +insert into t values ('08:00:00', '08:00:00'); +select t1.a, t2.b from t as t1 right join t as t2 on t1.a = t2.b; +a b +08:00:00 08:00:00 +select coercibility(1) ; +coercibility(1) +5 +select coercibility(null) ; +coercibility(null) +6 +select coercibility('abc') ; +coercibility('abc') +4 +select coercibility(version()) ; +coercibility(version()) +3 +select coercibility(user()) ; +coercibility(user()) +3 +select coercibility(database()) ; +coercibility(database()) +3 +select coercibility(current_role()) ; +coercibility(current_role()) +3 +select coercibility(current_user()) ; +coercibility(current_user()) +3 +select coercibility(1+null) ; +coercibility(1+null) +5 +select coercibility(null+'abcde') ; +coercibility(null+'abcde') +5 +select coercibility(concat(null, 'abcde')) ; +coercibility(concat(null, 'abcde')) +4 +select coercibility(rand()) ; +coercibility(rand()) +5 +select coercibility(now()) ; +coercibility(now()) +5 +select coercibility(sysdate()) ; +coercibility(sysdate()) +5 +drop table if exists t; +create table t (i int, r real, d datetime, t timestamp, c char(10), vc varchar(10), b binary(10), vb binary(10)); +insert into t values (null, null, null, null, null, null, null, null); +select coercibility(i) from t; +coercibility(i) +5 +select coercibility(r) from t; +coercibility(r) +5 +select coercibility(d) from t; +coercibility(d) +5 +select coercibility(t) from t; +coercibility(t) +5 +select coercibility(c) from t; +coercibility(c) +2 +select coercibility(b) from t; +coercibility(b) +2 +select coercibility(vb) from t; +coercibility(vb) +2 +select coercibility(vc) from t; +coercibility(vc) +2 +select coercibility(i+r) from t; +coercibility(i+r) +5 +select coercibility(i*r) from t; +coercibility(i*r) +5 +select coercibility(cos(r)+sin(i)) from t; +coercibility(cos(r)+sin(i)) +5 +select coercibility(d+2) from t; +coercibility(d+2) +5 +select coercibility(t*10) from t; +coercibility(t*10) +5 +select coercibility(concat(c, vc)) from t; +coercibility(concat(c, vc)) +2 +select coercibility(replace(c, 'x', 'y')) from t; +coercibility(replace(c, 'x', 'y')) +2 +SELECT COERCIBILITY(@straaa); +COERCIBILITY(@straaa) +2 +drop table if exists charset_test; +create table charset_test(id int auto_increment primary key, c1 varchar(255) character set ascii); +insert into charset_test(c1) values ('aaa�abcdef'); +Error 1366 (HY000): Incorrect string value '\xEF\xBF\xBD' for column 'c1' +insert into charset_test(c1) values ('aaa�'); +Error 1366 (HY000): Incorrect string value '\xEF\xBF\xBD' for column 'c1' +drop table if exists t_ci; +create table t_ci(a varchar(10) collate utf8mb4_general_ci, unique key(a)); +insert into t_ci values ('a'); +select * from t_ci; +a +a +select * from t_ci; +a +a +select * from t_ci where a='a'; +a +a +select * from t_ci where a='A'; +a +a +select * from t_ci where a='a '; +a +a +select * from t_ci where a='a '; +a +a +drop table if exists t; +create table t (a varchar(10) primary key,b int); +insert into t values ('a', 1), ('b', 3), ('a', 2) on duplicate key update b = b + 1; +set autocommit=0; +insert into t values ('a', 1), ('b', 3), ('a', 2) on duplicate key update b = b + 1; +select * from t; +a b +a 4 +b 4 +set autocommit=1; +select * from t; +a b +a 4 +b 4 +drop table if exists t; +create table t (a varchar(10),b int, key tk (a)); +insert into t values ('', 1), ('', 3); +set autocommit=0; +update t set b = b + 1; +select * from t; +a b + 2 + 4 +set autocommit=1; +select * from t; +a b + 2 + 4 +drop table t_ci; +create table t_ci(id bigint primary key, a varchar(10) collate utf8mb4_general_ci, unique key(a, id)); +insert into t_ci values (1, 'a'); +select a from t_ci; +a +a +select a from t_ci; +a +a +select a from t_ci where a='a'; +a +a +select a from t_ci where a='A'; +a +a +select a from t_ci where a='a '; +a +a +select a from t_ci where a='a '; +a +a +drop table if exists t; +create table t(c set('A', 'B') collate utf8mb4_general_ci); +insert into t values('a'); +insert into t values('B'); +select c from t where c = 'a'; +c +A +select c from t where c = 'A'; +c +A +select c from t where c = 'b'; +c +B +select c from t where c = 'B'; +c +B +drop table if exists t1; +CREATE TABLE `t1` ( `COL1` varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci NOT NULL, PRIMARY KEY (`COL1`(5)) clustered); +INSERT INTO `t1` VALUES ('Ȇ'); +select * from t1 where col1 not in (0xc484, 0xe5a4bc, 0xc3b3); +COL1 +Ȇ +select * from t1 where col1 >= 0xc484 and col1 <= 0xc3b3; +COL1 +Ȇ +select collation(IF('a' < 'B' collate utf8mb4_general_ci, 'smaller', 'greater' collate utf8mb4_unicode_ci)); +collation(IF('a' < 'B' collate utf8mb4_general_ci, 'smaller', 'greater' collate utf8mb4_unicode_ci)) +utf8mb4_unicode_ci +drop table if exists t; +create table t(a char(10)); +insert into t values ('a'); +select * from t where a in ('b' collate utf8mb4_general_ci, 'A', 3); +a +a +drop table if exists t; +create table t(`COL2` tinyint(16) DEFAULT NULL); +insert into t values(0); +select * from t WHERE COL2 IN (0xfc); +COL2 +select * from t WHERE COL2 = 0xfc; +COL2 +set autocommit=default; +drop table if exists t; +create table t (a varchar(10) collate utf8mb4_general_ci); +insert into t values ('a'); +insert into t values ('A'); +insert into t values ('b'); +insert into t values ('B'); +insert into t values ('a'); +insert into t values ('A'); +insert into t values ('ß'); +insert into t values ('sa'); +create index idx on t(a); +select * from t order by a; +a +a +A +a +A +b +B +ß +sa +drop table if exists t; +create table t (a varchar(10) collate utf8mb4_unicode_ci); +insert into t values ('a'); +insert into t values ('A'); +insert into t values ('b'); +insert into t values ('B'); +insert into t values ('a'); +insert into t values ('A'); +insert into t values ('ß'); +insert into t values ('sa'); +create index idx on t(a); +select * from t order by a; +a +a +A +a +A +b +B +sa +ß +select 'a' collate utf8mb4_bin = 'a' collate utf8mb4_general_ci; +Error 1267 (HY000): Illegal mix of collations (utf8mb4_bin,EXPLICIT) and (utf8mb4_general_ci,EXPLICIT) for operation '=' +drop table if exists t; +create table t ( +mb4general varchar(10) charset utf8mb4 collate utf8mb4_general_ci, +mb4unicode varchar(10) charset utf8mb4 collate utf8mb4_unicode_ci, +mb4bin varchar(10) charset utf8mb4 collate utf8mb4_bin, +general varchar(10) charset utf8 collate utf8_general_ci, +unicode varchar(10) charset utf8 collate utf8_unicode_ci, +utfbin varchar(10) charset utf8 collate utf8_bin, +bin varchar(10) charset binary collate binary, +latin1_bin varchar(10) charset latin1 collate latin1_bin, +ascii_bin varchar(10) charset ascii collate ascii_bin, +i int +); +insert into t values ('s', 's', 's', 's', 's', 's', 's', 's', 's', 1); +set names utf8mb4 collate utf8mb4_general_ci; +select * from t where mb4unicode = 's' collate utf8mb4_unicode_ci; +mb4general mb4unicode mb4bin general unicode utfbin bin latin1_bin ascii_bin i +s s s s s s s s s 1 +select * from t t1, t t2 where t1.mb4unicode = t2.mb4general collate utf8mb4_general_ci; +mb4general mb4unicode mb4bin general unicode utfbin bin latin1_bin ascii_bin i mb4general mb4unicode mb4bin general unicode utfbin bin latin1_bin ascii_bin i +s s s s s s s s s 1 s s s s s s s s s 1 +select * from t t1, t t2 where t1.mb4general = t2.mb4unicode collate utf8mb4_general_ci; +mb4general mb4unicode mb4bin general unicode utfbin bin latin1_bin ascii_bin i mb4general mb4unicode mb4bin general unicode utfbin bin latin1_bin ascii_bin i +s s s s s s s s s 1 s s s s s s s s s 1 +select * from t t1, t t2 where t1.mb4general = t2.mb4unicode collate utf8mb4_unicode_ci; +mb4general mb4unicode mb4bin general unicode utfbin bin latin1_bin ascii_bin i mb4general mb4unicode mb4bin general unicode utfbin bin latin1_bin ascii_bin i +s s s s s s s s s 1 s s s s s s s s s 1 +select * from t t1, t t2 where t1.mb4unicode = t2.mb4general collate utf8mb4_unicode_ci; +mb4general mb4unicode mb4bin general unicode utfbin bin latin1_bin ascii_bin i mb4general mb4unicode mb4bin general unicode utfbin bin latin1_bin ascii_bin i +s s s s s s s s s 1 s s s s s s s s s 1 +select * from t where mb4general = mb4bin collate utf8mb4_general_ci; +mb4general mb4unicode mb4bin general unicode utfbin bin latin1_bin ascii_bin i +s s s s s s s s s 1 +select * from t where mb4unicode = mb4general collate utf8mb4_unicode_ci; +mb4general mb4unicode mb4bin general unicode utfbin bin latin1_bin ascii_bin i +s s s s s s s s s 1 +select * from t where mb4general = mb4unicode collate utf8mb4_unicode_ci; +mb4general mb4unicode mb4bin general unicode utfbin bin latin1_bin ascii_bin i +s s s s s s s s s 1 +select * from t where mb4unicode = 's' collate utf8mb4_unicode_ci; +mb4general mb4unicode mb4bin general unicode utfbin bin latin1_bin ascii_bin i +s s s s s s s s s 1 +select * from t where mb4unicode = mb4bin; +mb4general mb4unicode mb4bin general unicode utfbin bin latin1_bin ascii_bin i +s s s s s s s s s 1 +select * from t where general = mb4unicode; +mb4general mb4unicode mb4bin general unicode utfbin bin latin1_bin ascii_bin i +s s s s s s s s s 1 +select * from t where unicode = mb4unicode; +mb4general mb4unicode mb4bin general unicode utfbin bin latin1_bin ascii_bin i +s s s s s s s s s 1 +select * from t where mb4unicode = mb4unicode; +mb4general mb4unicode mb4bin general unicode utfbin bin latin1_bin ascii_bin i +s s s s s s s s s 1 +select collation(concat(mb4unicode, mb4general collate utf8mb4_unicode_ci)) from t; +collation(concat(mb4unicode, mb4general collate utf8mb4_unicode_ci)) +utf8mb4_unicode_ci +select collation(concat(mb4general, mb4unicode, mb4bin)) from t; +collation(concat(mb4general, mb4unicode, mb4bin)) +utf8mb4_bin +select coercibility(concat(mb4general, mb4unicode, mb4bin)) from t; +coercibility(concat(mb4general, mb4unicode, mb4bin)) +1 +select collation(concat(mb4unicode, mb4bin, concat(mb4general))) from t; +collation(concat(mb4unicode, mb4bin, concat(mb4general))) +utf8mb4_bin +select coercibility(concat(mb4unicode, mb4bin)) from t; +coercibility(concat(mb4unicode, mb4bin)) +2 +select collation(concat(mb4unicode, mb4bin)) from t; +collation(concat(mb4unicode, mb4bin)) +utf8mb4_bin +select coercibility(concat(mb4bin, concat(mb4general))) from t; +coercibility(concat(mb4bin, concat(mb4general))) +2 +select collation(concaT(mb4bin, cOncAt(mb4general))) from t; +collation(concaT(mb4bin, cOncAt(mb4general))) +utf8mb4_bin +select coercibility(concat(mb4unicode, mb4bin, concat(mb4general))) from t; +coercibility(concat(mb4unicode, mb4bin, concat(mb4general))) +2 +select collation(concat(mb4unicode, mb4bin, concat(mb4general))) from t; +collation(concat(mb4unicode, mb4bin, concat(mb4general))) +utf8mb4_bin +select coercibility(concat(mb4unicode, mb4general)) from t; +coercibility(concat(mb4unicode, mb4general)) +1 +select collation(coalesce(mb4unicode, mb4general)) from t; +collation(coalesce(mb4unicode, mb4general)) +utf8mb4_bin +select coercibility(coalesce(mb4unicode, mb4general)) from t; +coercibility(coalesce(mb4unicode, mb4general)) +1 +select collation(CONCAT(concat(mb4unicode), concat(mb4general))) from t; +collation(CONCAT(concat(mb4unicode), concat(mb4general))) +utf8mb4_bin +select coercibility(cONcat(unicode, general)) from t; +coercibility(cONcat(unicode, general)) +1 +select collation(concAt(unicode, general)) from t; +collation(concAt(unicode, general)) +utf8_bin +select collation(concat(bin, mb4general)) from t; +collation(concat(bin, mb4general)) +binary +select coercibility(concat(bin, mb4general)) from t; +coercibility(concat(bin, mb4general)) +2 +select collation(concat(mb4unicode, ascii_bin)) from t; +collation(concat(mb4unicode, ascii_bin)) +utf8mb4_unicode_ci +select coercibility(concat(mb4unicode, ascii_bin)) from t; +coercibility(concat(mb4unicode, ascii_bin)) +2 +select collation(concat(mb4unicode, mb4unicode)) from t; +collation(concat(mb4unicode, mb4unicode)) +utf8mb4_unicode_ci +select coercibility(concat(mb4unicode, mb4unicode)) from t; +coercibility(concat(mb4unicode, mb4unicode)) +2 +select collation(concat(bin, bin)) from t; +collation(concat(bin, bin)) +binary +select coercibility(concat(bin, bin)) from t; +coercibility(concat(bin, bin)) +2 +select collation(concat(latin1_bin, ascii_bin)) from t; +collation(concat(latin1_bin, ascii_bin)) +latin1_bin +select coercibility(concat(latin1_bin, ascii_bin)) from t; +coercibility(concat(latin1_bin, ascii_bin)) +2 +select collation(concat(mb4unicode, bin)) from t; +collation(concat(mb4unicode, bin)) +binary +select coercibility(concat(mb4unicode, bin)) from t; +coercibility(concat(mb4unicode, bin)) +2 +select collation(mb4general collate utf8mb4_unicode_ci) from t; +collation(mb4general collate utf8mb4_unicode_ci) +utf8mb4_unicode_ci +select coercibility(mb4general collate utf8mb4_unicode_ci) from t; +coercibility(mb4general collate utf8mb4_unicode_ci) +0 +select collation(concat(concat(mb4unicode, mb4general), concat(unicode, general))) from t; +collation(concat(concat(mb4unicode, mb4general), concat(unicode, general))) +utf8mb4_bin +select coercibility(concat(concat(mb4unicode, mb4general), concat(unicode, general))) from t; +coercibility(concat(concat(mb4unicode, mb4general), concat(unicode, general))) +1 +select collation(concat(i, 1)) from t; +collation(concat(i, 1)) +utf8mb4_general_ci +select coercibility(concat(i, 1)) from t; +coercibility(concat(i, 1)) +4 +select collation(concat(i, user())) from t; +collation(concat(i, user())) +utf8mb4_bin +select coercibility(concat(i, user())) from t; +coercibility(concat(i, user())) +3 +select * from t where mb4unicode = mb4general; +Error 1267 (HY000): Illegal mix of collations (utf8mb4_unicode_ci,IMPLICIT) and (utf8mb4_general_ci,IMPLICIT) for operation '=' +select * from t where unicode = general; +Error 1267 (HY000): Illegal mix of collations (utf8_unicode_ci,IMPLICIT) and (utf8_general_ci,IMPLICIT) for operation '=' +select concat(mb4general) = concat(mb4unicode) from t; +Error 1267 (HY000): Illegal mix of collations (utf8mb4_general_ci,IMPLICIT) and (utf8mb4_unicode_ci,IMPLICIT) for operation '=' +select * from t t1, t t2 where t1.mb4unicode = t2.mb4general; +Error 1267 (HY000): Illegal mix of collations (utf8mb4_unicode_ci,IMPLICIT) and (utf8mb4_general_ci,IMPLICIT) for operation '=' +select field('s', mb4general, mb4unicode, mb4bin) from t; +Error 1271 (HY000): Illegal mix of collations for operation 'field' +select concat(mb4unicode, mb4general) = mb4unicode from t; +Error 1267 (HY000): Illegal mix of collations (utf8mb4_bin,NONE) and (utf8mb4_unicode_ci,IMPLICIT) for operation '=' +set names default; +drop table if exists t; +create table t(name char(255) primary key, b int, c int, index idx(name), unique index uidx(name)); +insert into t values("aaaa", 1, 1), ("bbb", 2, 2), ("ccc", 3, 3); +admin check table t; +drop table if exists t; +set tidb_enable_clustered_index=ON; +create table t(d double primary key, a int, name varchar(255), index idx(name(2)), index midx(a, name)); +insert into t values(2.11, 1, "aa"), (-1, 0, "abcd"), (9.99, 0, "aaaa"); +select d from t use index(idx) where name="aa"; +d +2.11 +set tidb_enable_clustered_index=default; +drop table if exists t; +create table t (a varchar(2) binary, index (a)); +insert into t values ('a '); +select hex(a) from t; +hex(a) +6120 +select hex(a) from t use index (a); +hex(a) +6120 +drop table if exists t; +create table t(a varchar(10) binary); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` varchar(10) DEFAULT NULL +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +drop table if exists t; +create table t(a varchar(10) binary) collate utf8_general_ci; +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` varchar(10) COLLATE utf8_bin DEFAULT NULL +) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_general_ci +drop table if exists t; +create table t(a varchar(10) binary collate utf8_general_ci); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` varchar(10) CHARACTER SET utf8 COLLATE utf8_bin DEFAULT NULL +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +drop table if exists t; +create table t(a varchar(10) binary charset utf8 collate utf8_general_ci); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` varchar(10) CHARACTER SET utf8 COLLATE utf8_general_ci DEFAULT NULL +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +drop table if exists t; +create table t(a varchar(10) binary charset utf8mb4 collate utf8mb4_unicode_ci) charset utf8 collate utf8_general_ci; +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` varchar(10) CHARACTER SET utf8mb4 COLLATE utf8mb4_unicode_ci DEFAULT NULL +) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_general_ci +drop table if exists t; +create table t(a varchar(10) binary charset binary); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` varbinary(10) DEFAULT NULL +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +drop table if exists t; +create table t(a TINYTEXT collate UTF8MB4_GENERAL_CI, UNIQUE KEY `a`(`a`(10))); +insert into t (a) values ('A'); +select * from t t1 inner join t t2 on t1.a = t2.a where t1.a = 'A'; +a a +A A +update t set a = 'B'; +admin check table t; +drop table if exists t; +set tidb_enable_clustered_index=ON; +create table t(a int, b char(10) collate utf8mb4_bin, c char(10) collate utf8mb4_general_ci,d varchar(10) collate utf8mb4_bin, e varchar(10) collate utf8mb4_general_ci, f char(10) collate utf8mb4_unicode_ci, g varchar(10) collate utf8mb4_unicode_ci, primary key(a, b, c, d, e, f, g), key a(a), unique key ua(a), key b(b), unique key ub(b), key c(c), unique key uc(c),key d(d), unique key ud(d),key e(e), unique key ue(e), key f(f), key g(g), unique key uf(f), unique key ug(g)); +insert into t values (1, '啊 ', '啊 ', '啊 ', '啊 ', '啊 ', '啊 '); +select * from t; +a b c d e f g +1 啊 啊 啊 啊 啊 啊 +select * from t use index(a); +a b c d e f g +1 啊 啊 啊 啊 啊 啊 +select * from t use index(ua); +a b c d e f g +1 啊 啊 啊 啊 啊 啊 +select * from t use index(b); +a b c d e f g +1 啊 啊 啊 啊 啊 啊 +select * from t use index(ub); +a b c d e f g +1 啊 啊 啊 啊 啊 啊 +select * from t use index(c); +a b c d e f g +1 啊 啊 啊 啊 啊 啊 +select * from t use index(uc); +a b c d e f g +1 啊 啊 啊 啊 啊 啊 +select * from t use index(d); +a b c d e f g +1 啊 啊 啊 啊 啊 啊 +select * from t use index(ud); +a b c d e f g +1 啊 啊 啊 啊 啊 啊 +select * from t use index(e); +a b c d e f g +1 啊 啊 啊 啊 啊 啊 +select * from t use index(ue); +a b c d e f g +1 啊 啊 啊 啊 啊 啊 +select * from t use index(f); +a b c d e f g +1 啊 啊 啊 啊 啊 啊 +select * from t use index(uf); +a b c d e f g +1 啊 啊 啊 啊 啊 啊 +select * from t use index(g); +a b c d e f g +1 啊 啊 啊 啊 啊 啊 +select * from t use index(ug); +a b c d e f g +1 啊 啊 啊 啊 啊 啊 +alter table t add column h varchar(10) collate utf8mb4_general_ci default '🐸'; +alter table t add column i varchar(10) collate utf8mb4_general_ci default '🐸'; +alter table t add index h(h); +alter table t add unique index uh(h); +select * from t use index(h); +a b c d e f g h i +1 啊 啊 啊 啊 啊 啊 🐸 🐸 +select * from t use index(uh); +a b c d e f g h i +1 啊 啊 啊 啊 啊 啊 🐸 🐸 +select * from t use index(a); +a b c d e f g h i +1 啊 啊 啊 啊 啊 啊 🐸 🐸 +select * from t use index(ua); +a b c d e f g h i +1 啊 啊 啊 啊 啊 啊 🐸 🐸 +select * from t use index(b); +a b c d e f g h i +1 啊 啊 啊 啊 啊 啊 🐸 🐸 +select * from t use index(ub); +a b c d e f g h i +1 啊 啊 啊 啊 啊 啊 🐸 🐸 +select * from t use index(c); +a b c d e f g h i +1 啊 啊 啊 啊 啊 啊 🐸 🐸 +select * from t use index(uc); +a b c d e f g h i +1 啊 啊 啊 啊 啊 啊 🐸 🐸 +select * from t use index(d); +a b c d e f g h i +1 啊 啊 啊 啊 啊 啊 🐸 🐸 +select * from t use index(ud); +a b c d e f g h i +1 啊 啊 啊 啊 啊 啊 🐸 🐸 +select * from t use index(e); +a b c d e f g h i +1 啊 啊 啊 啊 啊 啊 🐸 🐸 +select * from t use index(ue); +a b c d e f g h i +1 啊 啊 啊 啊 啊 啊 🐸 🐸 +admin check table t; +admin recover index t a; +ADDED_COUNT SCAN_COUNT +0 1 +alter table t add column n char(10) COLLATE utf8mb4_unicode_ci; +alter table t add index n(n); +update t set n = '吧'; +select * from t; +a b c d e f g h i n +1 啊 啊 啊 啊 啊 啊 🐸 🐸 吧 +select * from t use index(n); +a b c d e f g h i n +1 啊 啊 啊 啊 啊 啊 🐸 🐸 吧 +admin check table t; +drop table if exists t; +create table t (a varchar(255) COLLATE utf8_general_ci primary key clustered, b int) partition by range columns(a) (partition p0 values less than ('0'), partition p1 values less than MAXVALUE); +alter table t add index b(b); +insert into t values ('0', 1); +select * from t use index(b); +a b +0 1 +select * from t use index(); +a b +0 1 +admin check table t; +set tidb_enable_clustered_index=default; +drop table if exists t; +set tidb_enable_clustered_index=ON; +CREATE TABLE `t` (`a` char(10) COLLATE utf8mb4_unicode_ci NOT NULL,`b` char(20) COLLATE utf8mb4_general_ci NOT NULL,`c` int(11) NOT NULL,PRIMARY KEY (`a`,`b`,`c`),KEY `idx` (`a`)); +begin; +insert into t values ('a6', 'b6', 3); +select * from t; +a b c +a6 b6 3 +select * from t where a='a6'; +a b c +a6 b6 3 +delete from t; +select * from t; +a b c +commit; +select * from t; +a b c +drop table if exists t; +create table t(`a` char(10) COLLATE utf8mb4_unicode_ci NOT NULL key); +insert into t values ('&'); +replace into t values ('&'); +select * from t; +a +& +set tidb_enable_clustered_index=default; +drop table if exists t1, t2; +create table t1(a int, b char(10), key(b)) collate utf8mb4_general_ci; +create table t2(a int, b char(10), key(b)) collate ascii_bin; +insert into t1 values (1, 'a'); +insert into t2 values (1, 'A'); +select /*+ inl_join(t1) */ t1.b, t2.b from t1 join t2 where t1.b=t2.b; +b b +a A +select /*+ hash_join(t1) */ t1.b, t2.b from t1 join t2 where t1.b=t2.b; +b b +a A +select /*+ merge_join(t1) */ t1.b, t2.b from t1 join t2 where t1.b=t2.b; +b b +a A +select /*+ inl_hash_join(t1) */ t1.b, t2.b from t1 join t2 where t1.b=t2.b; +b b +a A +select /*+ inl_hash_join(t2) */ t1.b, t2.b from t1 join t2 where t1.b=t2.b; +b b +a A +Level Code Message +Warning 1815 Optimizer Hint /*+ INL_HASH_JOIN(t2) */ is inapplicable +select /*+ inl_merge_join(t1) */ t1.b, t2.b from t1 join t2 where t1.b=t2.b; +b b +a A +select /*+ inl_merge_join(t2) */ t1.b, t2.b from t1 join t2 where t1.b=t2.b; +b b +a A +Level Code Message +Warning 1815 Optimizer Hint /*+ INL_MERGE_JOIN(t2) */ is inapplicable +drop table if exists a, b; +create table a(i int, k varbinary(40), v int, primary key(i, k) clustered); +create table b(i int, k varchar(40), v int, primary key(i, k) clustered); +insert into a select 3, 'nice mccarthy', 10; +select * from a, b where a.i = b.i and a.k = b.k; +i k v i k v +drop table if exists a, b; +create table a(i int NOT NULL, k varbinary(40) NOT NULL, v int, key idx1(i, k)); +create table b(i int NOT NULL, k varchar(40) NOT NULL, v int, key idx1(i, k)); +insert into a select 3, 'nice mccarthy', 10; +select /*+ inl_join(b) */ b.i from a, b where a.i = b.i and a.k = b.k; +i +drop table if exists t; +CREATE TABLE `t` ( `col_10` blob DEFAULT NULL, `col_11` decimal(17,5) NOT NULL, `col_13` varchar(381) COLLATE utf8mb4_unicode_ci NOT NULL DEFAULT 'Yr', PRIMARY KEY (`col_13`,`col_11`) CLUSTERED, KEY `idx_4` (`col_10`(3))) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin; +insert into t values ('a', 12523, 'A'); +insert into t values ('A', 2, 'a'); +insert into t values ('a', 23, 'A'); +insert into t values ('a', 23, 'h2'); +insert into t values ('a', 23, 'h3'); +insert into t values ('a', 23, 'h4'); +insert into t values ('a', 23, 'h5'); +insert into t values ('a', 23, 'h6'); +insert into t values ('a', 23, 'h7'); +select /*+ MERGE_JOIN(t) */ t.* from t where col_13 in ( select col_10 from t where t.col_13 in ( 'a', 'b' ) ) order by col_10 ; +col_10 col_11 col_13 +A 2.00000 a +a 23.00000 A +a 12523.00000 A +select 'a' like 'A' collate utf8mb4_unicode_ci; +'a' like 'A' collate utf8mb4_unicode_ci +1 +select 'a' collate utf8mb4_bin like 'A' collate utf8mb4_unicode_ci; +Error 1267 (HY000): Illegal mix of collations (utf8mb4_bin,EXPLICIT) and (utf8mb4_unicode_ci,EXPLICIT) for operation 'like' +select '😛' collate utf8mb4_general_ci like '😋'; +'😛' collate utf8mb4_general_ci like '😋' +1 +select '😛' collate utf8mb4_general_ci = '😋'; +'😛' collate utf8mb4_general_ci = '😋' +1 +select '😛' collate utf8mb4_unicode_ci like '😋'; +'😛' collate utf8mb4_unicode_ci like '😋' +0 +select '😛' collate utf8mb4_unicode_ci = '😋'; +'😛' collate utf8mb4_unicode_ci = '😋' +1 +drop table if exists t; +create table t (k char(20), v int, primary key (k(4)) clustered, key (k)) collate utf8mb4_general_ci; +insert into t values('01233', 1); +create index idx on t(k(2)); +select * from t use index(k_2); +k v +01233 1 +select * from t use index(idx); +k v +01233 1 +admin check table t; +drop table if exists t; +CREATE TABLE t (`COL1` tinyblob NOT NULL, `COL2` binary(1) NOT NULL, `COL3` bigint(11) NOT NULL, PRIMARY KEY (`COL1`(5),`COL2`,`COL3`) /*T![clustered_index] CLUSTERED */); +insert into t values(0x1E,0xEC,6966939640596047133); +select * from t where col1 not in (0x1B,0x20) order by col1; +COL1 COL2 COL3 + 6966939640596047133 +drop table if exists t; +create table t(a varchar(10)); +insert into t values('aaaaaaaaa'),('天王盖地虎宝塔镇河妖'); +select * from t; +a +aaaaaaaaa +天王盖地虎宝塔镇河妖 +select collation(a) from (select null as a) aaa; +collation(a) +binary +select collation(a) from (select a from t limit 1) aaa; +collation(a) +utf8mb4_bin +select * from (select null as a union all select a from t) aaa order by a; +a +NULL +aaaaaaaaa +天王盖地虎宝塔镇河妖 +select * from (select a from t) aaa union all select null as a order by a; +a +NULL +aaaaaaaaa +天王盖地虎宝塔镇河妖 +drop table if exists t; +create table t (a char(10) collate utf8mb4_bin, b char(10) collate utf8mb4_general_ci); +insert into t values ('a', 'A'); +select * from t t1, t t2 where t1.a=t2.b and t2.b='a' collate utf8mb4_general_ci; +a b a b +select * from t t1, t t2 where t1.a=t2.b and t2.b>='a' collate utf8mb4_general_ci; +a b a b +drop table t; +create table t (a char(10) collate utf8mb4_general_ci, b char(10) collate utf8mb4_general_ci); +insert into t values ('A', 'a'); +select * from t t1, t t2 where t1.a=t2.b and t2.b='a' collate utf8mb4_bin; +a b a b +A a A a +select * from t t1, t t2 where t1.a=t2.b and t2.b>='a' collate utf8mb4_bin; +a b a b +A a A a +drop table t; +set names utf8mb4; +create table t (a char(10) collate utf8mb4_general_ci, b char(10) collate utf8_general_ci); +insert into t values ('a', 'A'); +select * from t t1, t t2 where t1.a=t2.b and t2.b='A'; +a b a b +a A a A +drop table t; +create table t(a char collate utf8_general_ci, b char collate utf8mb4_general_ci, c char collate utf8_bin); +insert into t values ('b', 'B', 'B'); +select * from t t1, t t2 where t1.a=t2.b and t2.b=t2.c; +a b c a b c +b B B b B B +drop table t; +create table t(a char collate utf8_bin, b char collate utf8_general_ci); +insert into t values ('a', 'A'); +select * from t t1, t t2 where t1.b=t2.b and t2.b=t1.a collate utf8_general_ci; +a b a b +a A a A +drop table if exists t1, t2; +set names utf8mb4 collate utf8mb4_general_ci; +create table t1(a char, b varchar(10)) charset utf8mb4 collate utf8mb4_general_ci; +create table t2(a char, b varchar(10)) charset utf8mb4 collate utf8mb4_bin; +insert into t1 values ('A', 'a'); +insert into t2 values ('a', 'a'); +select * from t1 left join t2 on t1.a = t2.a where t1.a = 'a'; +a b a b +A a NULL NULL +drop table t; +set names utf8mb4 collate utf8mb4_general_ci; +create table t(a char collate utf8mb4_bin, b char collate utf8mb4_general_ci); +insert into t values ('a', 'a'); +select * from t t1, t t2 where t2.b = 'A' and lower(concat(t1.a , '' )) = t2.b; +a b a b +a a a a +drop table t; +create table t(a char collate utf8_unicode_ci, b char collate utf8mb4_unicode_ci, c char collate utf8_bin); +insert into t values ('b', 'B', 'B'); +select * from t t1, t t2 where t1.a=t2.b and t2.b=t2.c; +a b c a b c +b B B b B B +drop table if exists t1, t2; +set names utf8mb4 collate utf8mb4_unicode_ci; +create table t1(a char, b varchar(10)) charset utf8mb4 collate utf8mb4_unicode_ci; +create table t2(a char, b varchar(10)) charset utf8mb4 collate utf8mb4_bin; +insert into t1 values ('A', 'a'); +insert into t2 values ('a', 'a'); +select * from t1 left join t2 on t1.a = t2.a where t1.a = 'a'; +a b a b +A a NULL NULL +drop table if exists t1, t2; +set names utf8mb4 collate utf8mb4_general_ci; +create table t1(a char, b varchar(10)) charset utf8mb4 collate utf8mb4_general_ci; +create table t2(a char, b varchar(10)) charset utf8mb4 collate utf8mb4_unicode_ci; +insert into t1 values ('ß', 's'); +insert into t2 values ('s', 's'); +select * from t1 left join t2 on t1.a = t2.a collate utf8mb4_unicode_ci where t1.a = 's'; +a b a b +ß s NULL NULL +drop table if exists t1, t2; +create table t1(a char(10) collate utf8mb4_general_ci, index (a)); +create table t2(a char(10) collate utf8_bin, index (a)); +insert into t1 values ('a'); +insert into t2 values ('A'); +set names utf8 collate utf8_general_ci; +select * from t1, t2 where t1.a=t2.a and t1.a= 'a'; +a a +a A +select * from t1 where a='a' and a = 'A'; +a +a +set names default; +drop table if exists t; +drop table if exists t_bin; +CREATE TABLE `t` ( `a` int(11) NOT NULL,`b` varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci DEFAULT NULL); +CREATE TABLE `t_bin` ( `a` int(11) NOT NULL,`b` varchar(5) CHARACTER SET binary); +insert into t values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' '); +insert into t_bin values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' '); +select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b order by t1.a; +a b +1 a +1 a +1 a +1 a +2 À +2 À +2 À +2 À +3 á +3 á +3 á +3 á +4 à +4 à +4 à +4 à +5 b +6 c +7 +select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b order by t1.a; +a b +1 a +2 À +3 á +4 à +5 b +6 c +7 +select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>3 order by t1.a; +a b +4 à +4 à +4 à +4 à +5 b +6 c +7 +select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>3 order by t1.a; +a b +4 à +5 b +6 c +7 +select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>3 order by t1.a; +a b +4 à +4 à +4 à +4 à +5 b +6 c +7 +select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>3 order by t1.a; +a b +4 à +5 b +6 c +7 +select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>t2.a order by t1.a; +a b +2 À +3 á +3 á +4 à +4 à +4 à +select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>t2.a order by t1.a; +a b +select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b order by t1.a; +a b +1 a +1 a +1 a +1 a +2 À +2 À +2 À +2 À +3 á +3 á +3 á +3 á +4 à +4 à +4 à +4 à +5 b +6 c +7 +select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b order by t1.a; +a b +1 a +2 À +3 á +4 à +5 b +6 c +7 +select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>3 order by t1.a; +a b +4 à +4 à +4 à +4 à +5 b +6 c +7 +select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>3 order by t1.a; +a b +4 à +5 b +6 c +7 +select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>3 order by t1.a; +a b +4 à +4 à +4 à +4 à +5 b +6 c +7 +select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>3 order by t1.a; +a b +4 à +5 b +6 c +7 +select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>t2.a order by t1.a; +a b +2 À +3 á +3 á +4 à +4 à +4 à +select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>t2.a order by t1.a; +a b +drop table if exists t1; +drop table if exists t2; +create table t1 (id int, v varchar(5) character set binary, key(v)); +create table t2 (v varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci, key(v)); +insert into t1 values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' '); +insert into t2 values ('a'), ('À'), ('á'), ('à'), ('b'), ('c'), (' '); +select /*+ TIDB_HJ(t1, t2) */ * from t1, t2 where t1.v=t2.v order by t1.id; +id v v +1 a a +2 À À +3 á á +4 à à +5 b b +6 c c +7 +select /*+ TIDB_SMJ(t1, t2) */ * from t1, t2 where t1.v=t2.v order by t1.id; +id v v +1 a a +2 À À +3 á á +4 à à +5 b b +6 c c +7 +drop table if exists t; +create table t (a varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci, b varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci, key(a), key(b)); +insert into t values ('a', 'x'), ('x', 'À'), ('á', 'x'), ('à', 'à'), ('à', 'x'); +select /*+ USE_INDEX_MERGE(t, a, b) */ * from t where a = 'a' or b = 'a'; +a b +a x +x À +à x +à à +á x +select field('a', 'b', 'a'); +field('a', 'b', 'a') +2 +select field('a', 'b', 'A'); +field('a', 'b', 'A') +0 +select field('a', 'b', 'A' collate utf8mb4_bin); +field('a', 'b', 'A' collate utf8mb4_bin) +0 +select field('a', 'b', 'a ' collate utf8mb4_bin); +field('a', 'b', 'a ' collate utf8mb4_bin) +2 +select field('a', 'b', 'A' collate utf8mb4_unicode_ci); +field('a', 'b', 'A' collate utf8mb4_unicode_ci) +2 +select field('a', 'b', 'a ' collate utf8mb4_unicode_ci); +field('a', 'b', 'a ' collate utf8mb4_unicode_ci) +2 +select field('a', 'b', 'A' collate utf8mb4_general_ci); +field('a', 'b', 'A' collate utf8mb4_general_ci) +2 +select field('a', 'b', 'a ' collate utf8mb4_general_ci); +field('a', 'b', 'a ' collate utf8mb4_general_ci) +2 +drop table if exists t; +create table t(a char(10), b char (10)) collate utf8mb4_general_ci; +insert into t values ('a', 'A'); +select field(a, b) from t; +field(a, b) +1 +select FIND_IN_SET('a','b,a,c,d'); +FIND_IN_SET('a','b,a,c,d') +2 +select FIND_IN_SET('a','b,A,c,d'); +FIND_IN_SET('a','b,A,c,d') +0 +select FIND_IN_SET('a','b,A,c,d' collate utf8mb4_bin); +FIND_IN_SET('a','b,A,c,d' collate utf8mb4_bin) +0 +select FIND_IN_SET('a','b,a ,c,d' collate utf8mb4_bin); +FIND_IN_SET('a','b,a ,c,d' collate utf8mb4_bin) +2 +select FIND_IN_SET('a','b,A,c,d' collate utf8mb4_general_ci); +FIND_IN_SET('a','b,A,c,d' collate utf8mb4_general_ci) +2 +select FIND_IN_SET('a','b,a ,c,d' collate utf8mb4_general_ci); +FIND_IN_SET('a','b,a ,c,d' collate utf8mb4_general_ci) +2 +set names utf8mb4 collate utf8mb4_general_ci; +select collation(cast('a' as char)); +collation(cast('a' as char)) +utf8mb4_general_ci +select collation(cast('a' as binary)); +collation(cast('a' as binary)) +binary +select collation(cast('a' collate utf8mb4_bin as char)); +collation(cast('a' collate utf8mb4_bin as char)) +utf8mb4_general_ci +select collation(cast('a' collate utf8mb4_bin as binary)); +collation(cast('a' collate utf8mb4_bin as binary)) +binary +select FIND_IN_SET('a','b,A,c,d' collate utf8mb4_unicode_ci); +FIND_IN_SET('a','b,A,c,d' collate utf8mb4_unicode_ci) +2 +select FIND_IN_SET('a','b,a ,c,d' collate utf8mb4_unicode_ci); +FIND_IN_SET('a','b,a ,c,d' collate utf8mb4_unicode_ci) +2 +select concat('a' collate utf8mb4_bin, 'b' collate utf8mb4_bin); +concat('a' collate utf8mb4_bin, 'b' collate utf8mb4_bin) +ab +select concat('a' collate utf8mb4_bin, 'b' collate utf8mb4_general_ci); +Error 1267 (HY000): Illegal mix of collations (utf8mb4_bin,EXPLICIT) and (utf8mb4_general_ci,EXPLICIT) for operation 'concat' +drop table if exists t; +create table t(a char); +select * from t t1 join t t2 on t1.a collate utf8mb4_bin = t2.a collate utf8mb4_general_ci; +Error 1267 (HY000): Illegal mix of collations (utf8mb4_bin,EXPLICIT) and (utf8mb4_general_ci,EXPLICIT) for operation '=' +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 ( a int, p1 VARCHAR(255) CHARACTER SET utf8 COLLATE utf8_bin,p2 VARCHAR(255) CHARACTER SET utf8 COLLATE utf8_general_ci , p3 VARCHAR(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin,p4 VARCHAR(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci ,n1 VARCHAR(255) CHARACTER SET utf8 COLLATE utf8_bin,n2 VARCHAR(255) CHARACTER SET utf8 COLLATE utf8_general_ci , n3 VARCHAR(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin,n4 VARCHAR(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci ); +insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values(1,' 0aA1!测试テストמבחן ',' 0aA1!测试テストמבחן ',' 0aA1!测试テストמבחן ',' 0aA1!测试テストמבחן ',' 0Aa1!测试テストמבחן ',' 0Aa1!测试テストמבחן ',' 0Aa1!测试テストמבחן ',' 0Aa1!测试テストמבחן '); +select INSTR(p1,n1) from t1; +INSTR(p1,n1) +0 +select INSTR(p1,n2) from t1; +INSTR(p1,n2) +0 +select INSTR(p1,n3) from t1; +INSTR(p1,n3) +0 +select INSTR(p1,n4) from t1; +INSTR(p1,n4) +0 +select INSTR(p2,n1) from t1; +INSTR(p2,n1) +0 +select INSTR(p2,n2) from t1; +INSTR(p2,n2) +1 +select INSTR(p2,n3) from t1; +INSTR(p2,n3) +0 +select INSTR(p2,n4) from t1; +INSTR(p2,n4) +1 +select INSTR(p3,n1) from t1; +INSTR(p3,n1) +0 +select INSTR(p3,n2) from t1; +INSTR(p3,n2) +0 +select INSTR(p3,n3) from t1; +INSTR(p3,n3) +0 +select INSTR(p3,n4) from t1; +INSTR(p3,n4) +0 +select INSTR(p4,n1) from t1; +INSTR(p4,n1) +0 +select INSTR(p4,n2) from t1; +INSTR(p4,n2) +1 +select INSTR(p4,n3) from t1; +INSTR(p4,n3) +0 +select INSTR(p4,n4) from t1; +INSTR(p4,n4) +1 +truncate table t1; +insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (1,'0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן'); +insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (2,'0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן'); +insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (3,'0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0Aa1!测试テストמבחן ','0Aa1!测试テストמבחן ','0Aa1!测试テストמבחן ','0Aa1!测试テストמבחן '); +select LOCATE(p1,n1) from t1; +LOCATE(p1,n1) +0 +0 +0 +select LOCATE(p1,n2) from t1; +LOCATE(p1,n2) +0 +0 +0 +select LOCATE(p1,n3) from t1; +LOCATE(p1,n3) +0 +0 +0 +select LOCATE(p1,n4) from t1; +LOCATE(p1,n4) +0 +1 +1 +select LOCATE(p2,n1) from t1; +LOCATE(p2,n1) +0 +0 +0 +select LOCATE(p2,n2) from t1; +LOCATE(p2,n2) +0 +1 +1 +select LOCATE(p2,n3) from t1; +LOCATE(p2,n3) +0 +0 +0 +select LOCATE(p2,n4) from t1; +LOCATE(p2,n4) +0 +1 +1 +select LOCATE(p3,n1) from t1; +LOCATE(p3,n1) +0 +0 +0 +select LOCATE(p3,n2) from t1; +LOCATE(p3,n2) +0 +0 +0 +select LOCATE(p3,n3) from t1; +LOCATE(p3,n3) +0 +0 +0 +select LOCATE(p3,n4) from t1; +LOCATE(p3,n4) +0 +0 +0 +select LOCATE(p4,n1) from t1; +LOCATE(p4,n1) +0 +1 +1 +select LOCATE(p4,n2) from t1; +LOCATE(p4,n2) +0 +1 +1 +select LOCATE(p4,n3) from t1; +LOCATE(p4,n3) +0 +0 +0 +select LOCATE(p4,n4) from t1; +LOCATE(p4,n4) +0 +1 +1 +select locate('S', 's' collate utf8mb4_general_ci); +locate('S', 's' collate utf8mb4_general_ci) +1 +select locate('S', 'a' collate utf8mb4_general_ci); +locate('S', 'a' collate utf8mb4_general_ci) +0 +select locate('ß', 's' collate utf8mb4_general_ci); +locate('ß', 's' collate utf8mb4_general_ci) +1 +select locate('world', 'hello world' collate utf8mb4_general_ci); +locate('world', 'hello world' collate utf8mb4_general_ci) +7 +select locate(' ', 'hello world' collate utf8mb4_general_ci); +locate(' ', 'hello world' collate utf8mb4_general_ci) +6 +select locate(' ', 'hello world' collate utf8mb4_general_ci); +locate(' ', 'hello world' collate utf8mb4_general_ci) +0 +select locate('S', 's' collate utf8mb4_unicode_ci); +locate('S', 's' collate utf8mb4_unicode_ci) +1 +select locate('S', 'a' collate utf8mb4_unicode_ci); +locate('S', 'a' collate utf8mb4_unicode_ci) +0 +select locate('ß', 'ss' collate utf8mb4_unicode_ci); +locate('ß', 'ss' collate utf8mb4_unicode_ci) +1 +select locate('world', 'hello world' collate utf8mb4_unicode_ci); +locate('world', 'hello world' collate utf8mb4_unicode_ci) +7 +select locate(' ', 'hello world' collate utf8mb4_unicode_ci); +locate(' ', 'hello world' collate utf8mb4_unicode_ci) +6 +select locate(' ', 'hello world' collate utf8mb4_unicode_ci); +locate(' ', 'hello world' collate utf8mb4_unicode_ci) +0 +truncate table t1; +insert into t1 (a) values (1); +insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (2,'0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן'); +insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (3,'0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן'); +insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (4,'0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0Aa1!测试テストמבחן ','0Aa1!测试テストמבחן ','0Aa1!测试テストמבחן ','0Aa1!测试テストמבחן '); +insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (5,'0aA1!测试テストמבחן0aA1!测试','0aA1!测试テストמבחן0aA1!测试','0aA1!测试テストמבחן0aA1!测试','0aA1!测试テストמבחן0aA1!测试','0Aa1!测试','0Aa1!测试','0Aa1!测试','0Aa1!测试'); +insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (6,'0aA1!测试テストמבחן0aA1!测试','0aA1!测试テストמבחן0aA1!测试','0aA1!测试テストמבחן0aA1!测试','0aA1!测试テストמבחן0aA1!测试','0aA1!测试','0aA1!测试','0aA1!测试','0aA1!测试'); +insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (7,'0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן'); +insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (8,'0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן '); +select p1 REGEXP n1 from t1; +p1 REGEXP n1 +NULL +0 +0 +0 +0 +1 +1 +0 +select p1 REGEXP n2 from t1; +p1 REGEXP n2 +NULL +0 +0 +0 +0 +1 +1 +0 +select p1 REGEXP n3 from t1; +p1 REGEXP n3 +NULL +0 +0 +0 +0 +1 +1 +0 +select p1 REGEXP n4 from t1; +p1 REGEXP n4 +NULL +1 +1 +0 +1 +1 +1 +0 +select p2 REGEXP n1 from t1; +p2 REGEXP n1 +NULL +0 +0 +0 +0 +1 +1 +0 +select p2 REGEXP n2 from t1; +p2 REGEXP n2 +NULL +1 +1 +0 +1 +1 +1 +0 +select p2 REGEXP n3 from t1; +p2 REGEXP n3 +NULL +0 +0 +0 +0 +1 +1 +0 +select p2 REGEXP n4 from t1; +p2 REGEXP n4 +NULL +1 +1 +0 +1 +1 +1 +0 +select p3 REGEXP n1 from t1; +p3 REGEXP n1 +NULL +0 +0 +0 +0 +1 +1 +0 +select p3 REGEXP n2 from t1; +p3 REGEXP n2 +NULL +0 +0 +0 +0 +1 +1 +0 +select p3 REGEXP n3 from t1; +p3 REGEXP n3 +NULL +0 +0 +0 +0 +1 +1 +0 +select p3 REGEXP n4 from t1; +p3 REGEXP n4 +NULL +0 +0 +0 +0 +1 +1 +0 +select p4 REGEXP n1 from t1; +p4 REGEXP n1 +NULL +1 +1 +0 +1 +1 +1 +0 +select p4 REGEXP n2 from t1; +p4 REGEXP n2 +NULL +1 +1 +0 +1 +1 +1 +0 +select p4 REGEXP n3 from t1; +p4 REGEXP n3 +NULL +0 +0 +0 +0 +1 +1 +0 +select p4 REGEXP n4 from t1; +p4 REGEXP n4 +NULL +1 +1 +0 +1 +1 +1 +0 +set names default; +set names utf8mb4 collate utf8mb4_general_ci; +select 'a' like 'A'; +'a' like 'A' +1 +select 'a' like 'A' collate utf8mb4_general_ci; +'a' like 'A' collate utf8mb4_general_ci +1 +select 'a' like 'À'; +'a' like 'À' +1 +select 'a' like '%À'; +'a' like '%À' +1 +select 'a' like '%À '; +'a' like '%À ' +0 +select 'a' like 'À%'; +'a' like 'À%' +1 +select 'a' like 'À_'; +'a' like 'À_' +0 +select 'a' like '%À%'; +'a' like '%À%' +1 +select 'aaa' like '%ÀAa%'; +'aaa' like '%ÀAa%' +1 +set names utf8mb4 collate utf8mb4_bin; +drop table if exists t_like; +create table t_like(id int, b varchar(20) collate utf8mb4_general_ci); +insert into t_like values (1, 'aaa'), (2, 'abc'), (3, 'aac'); +select b like 'AaÀ' from t_like order by id; +b like 'AaÀ' +1 +0 +0 +select b like 'Aa_' from t_like order by id; +b like 'Aa_' +1 +0 +1 +select b like '_A_' from t_like order by id; +b like '_A_' +1 +0 +1 +select b from t_like where b like 'Aa_' order by id; +b +aaa +aac +select b from t_like where b like 'A%' order by id; +b +aaa +abc +aac +select b from t_like where b like '%A%' order by id; +b +aaa +abc +aac +alter table t_like add index idx_b(b); +select b from t_like use index(idx_b) where b like 'Aa_' order by id; +b +aaa +aac +select b from t_like use index(idx_b) where b like 'A%' order by id; +b +aaa +abc +aac +select b from t_like use index(idx_b) where b like '%A%' order by id; +b +aaa +abc +aac +set names default; +drop table if exists t; +drop table if exists t_bin; +create table t (id int, v varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci DEFAULT NULL ); +create table t_bin (id int, v varchar(5) CHARACTER SET binary ); +insert into t values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' '); +insert into t_bin values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' '); +select v from t where v='a' order by id; +v +a +À +á +à +select v from t_bin where v='a' order by id; +v +a +select v from t where v<'b' and id<=3; +v +a +À +á +select v from t_bin where v<'b' and id<=3; +v +a +select id from t order by v, id; +id +7 +1 +2 +3 +4 +5 +6 +select id from t_bin order by v, id; +id +7 +1 +5 +6 +2 +4 +3 +explain format="brief" select distinct(v) from t_bin; +id estRows task access object operator info +HashAgg 8000.00 root group by:expression__charset_and_collation.t_bin.v, funcs:firstrow(expression__charset_and_collation.t_bin.v)->expression__charset_and_collation.t_bin.v +└─TableReader 8000.00 root data:HashAgg + └─HashAgg 8000.00 cop[tikv] group by:expression__charset_and_collation.t_bin.v, + └─TableFullScan 10000.00 cop[tikv] table:t_bin keep order:false, stats:pseudo +select distinct(v) from t_bin; +v + +a +b +c +À +à +á +explain format="brief" select distinct(v) from t; +id estRows task access object operator info +HashAgg 8000.00 root group by:expression__charset_and_collation.t.v, funcs:firstrow(expression__charset_and_collation.t.v)->expression__charset_and_collation.t.v +└─TableReader 8000.00 root data:HashAgg + └─HashAgg 8000.00 cop[tikv] group by:expression__charset_and_collation.t.v, + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select distinct(v) from t; +v + +a +b +c +explain format="brief" select v, count(*) from t_bin group by v; +id estRows task access object operator info +Projection 8000.00 root expression__charset_and_collation.t_bin.v, Column#4 +└─HashAgg 8000.00 root group by:expression__charset_and_collation.t_bin.v, funcs:count(Column#5)->Column#4, funcs:firstrow(expression__charset_and_collation.t_bin.v)->expression__charset_and_collation.t_bin.v + └─TableReader 8000.00 root data:HashAgg + └─HashAgg 8000.00 cop[tikv] group by:expression__charset_and_collation.t_bin.v, funcs:count(1)->Column#5 + └─TableFullScan 10000.00 cop[tikv] table:t_bin keep order:false, stats:pseudo +select v, count(*) from t_bin group by v; +v count(*) + 1 +a 1 +b 1 +c 1 +À 1 +à 1 +á 1 +explain format="brief" select v, count(*) from t group by v; +id estRows task access object operator info +Projection 8000.00 root expression__charset_and_collation.t.v, Column#4 +└─HashAgg 8000.00 root group by:expression__charset_and_collation.t.v, funcs:count(Column#5)->Column#4, funcs:firstrow(expression__charset_and_collation.t.v)->expression__charset_and_collation.t.v + └─TableReader 8000.00 root data:HashAgg + └─HashAgg 8000.00 cop[tikv] group by:expression__charset_and_collation.t.v, funcs:count(1)->Column#5 + └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select v, count(*) from t group by v; +v count(*) + 1 +a 4 +b 1 +c 1 +drop table if exists t; +create table t(a char(10) collate utf8mb4_general_ci, key(a)); +insert into t values ('a'), ('A'), ('b'); +insert into t values ('a'), ('A'), ('b'); +insert into t values ('a'), ('A'), ('b'); +select * from t order by a collate utf8mb4_bin; +a +A +A +A +a +a +a +b +b +b +select * from t order by a collate utf8mb4_general_ci; +a +a +A +a +A +a +A +b +b +b +select * from t order by a collate utf8mb4_unicode_ci; +a +a +A +a +A +a +A +b +b +b +drop table if exists t; +create table t(a char(10) collate utf8mb4_general_ci, key(a)); +insert into t values ('a'), ('A'), ('b'); +insert into t values ('a'), ('A'), ('b'); +insert into t values ('a'), ('A'), ('b'); +insert into t values ('s'), ('ss'), ('ß'); +select count(1) from t group by a collate utf8mb4_bin order by a collate utf8mb4_bin; +count(1) +3 +3 +3 +1 +1 +1 +select count(1) from t group by a collate utf8mb4_unicode_ci order by a collate utf8mb4_unicode_ci; +count(1) +6 +3 +1 +2 +select count(1) from t group by a collate utf8mb4_general_ci order by a collate utf8mb4_general_ci; +count(1) +6 +3 +2 +1 +drop table if exists t; +drop table if exists t_bin; +create table t (id int, v varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci DEFAULT NULL , key(v)); +create table t_bin (id int, v varchar(5) CHARACTER SET binary , key(v)); +insert into t values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' '); +insert into t_bin values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' '); +explain format="brief" select /*+ STREAM_AGG() */ distinct(v) from t_bin; +id estRows task access object operator info +StreamAgg 8000.00 root group by:expression__charset_and_collation.t_bin.v, funcs:firstrow(expression__charset_and_collation.t_bin.v)->expression__charset_and_collation.t_bin.v +└─IndexReader 8000.00 root index:StreamAgg + └─StreamAgg 8000.00 cop[tikv] group by:expression__charset_and_collation.t_bin.v, + └─IndexFullScan 10000.00 cop[tikv] table:t_bin, index:v(v) keep order:true, stats:pseudo +select /*+ STREAM_AGG() */ distinct(v) from t_bin; +v + +a +b +c +À +à +á +explain format="brief" select /*+ STREAM_AGG() */ distinct(v) from t; +id estRows task access object operator info +StreamAgg 8000.00 root group by:expression__charset_and_collation.t.v, funcs:firstrow(expression__charset_and_collation.t.v)->expression__charset_and_collation.t.v +└─IndexReader 8000.00 root index:StreamAgg + └─StreamAgg 8000.00 cop[tikv] group by:expression__charset_and_collation.t.v, + └─IndexFullScan 10000.00 cop[tikv] table:t, index:v(v) keep order:true, stats:pseudo +select /*+ STREAM_AGG() */ distinct(v) from t; +v + +a +b +c +explain format="brief" select /*+ STREAM_AGG() */ v, count(*) from t_bin group by v; +id estRows task access object operator info +Projection 8000.00 root expression__charset_and_collation.t_bin.v, Column#4 +└─StreamAgg 8000.00 root group by:expression__charset_and_collation.t_bin.v, funcs:count(Column#5)->Column#4, funcs:firstrow(expression__charset_and_collation.t_bin.v)->expression__charset_and_collation.t_bin.v + └─IndexReader 8000.00 root index:StreamAgg + └─StreamAgg 8000.00 cop[tikv] group by:expression__charset_and_collation.t_bin.v, funcs:count(1)->Column#5 + └─IndexFullScan 10000.00 cop[tikv] table:t_bin, index:v(v) keep order:true, stats:pseudo +select /*+ STREAM_AGG() */ v, count(*) from t_bin group by v; +v count(*) + 1 +a 1 +b 1 +c 1 +À 1 +à 1 +á 1 +explain format="brief" select /*+ STREAM_AGG() */ v, count(*) from t group by v; +id estRows task access object operator info +Projection 8000.00 root expression__charset_and_collation.t.v, Column#4 +└─StreamAgg 8000.00 root group by:expression__charset_and_collation.t.v, funcs:count(Column#5)->Column#4, funcs:firstrow(expression__charset_and_collation.t.v)->expression__charset_and_collation.t.v + └─IndexReader 8000.00 root index:StreamAgg + └─StreamAgg 8000.00 cop[tikv] group by:expression__charset_and_collation.t.v, funcs:count(1)->Column#5 + └─IndexFullScan 10000.00 cop[tikv] table:t, index:v(v) keep order:true, stats:pseudo +select /*+ STREAM_AGG() */ v, count(*) from t group by v; +v count(*) + 1 +a 4 +b 1 +c 1 +explain format="brief" select v from t where v < 'b' order by v; +id estRows task access object operator info +IndexReader 3323.33 root index:IndexRangeScan +└─IndexRangeScan 3323.33 cop[tikv] table:t, index:v(v) range:[-inf,"\x00B"), keep order:true, stats:pseudo +select v from t where v < 'b' order by v; +v + +a +À +á +à +explain format="brief" select v from t where v < 'b' and v > ' ' order by v; +id estRows task access object operator info +IndexReader 250.00 root index:IndexRangeScan +└─IndexRangeScan 250.00 cop[tikv] table:t, index:v(v) range:("","\x00B"), keep order:true, stats:pseudo +select v from t where v < 'b' and v > ' ' order by v; +v +a +À +á +à +explain format="brief" select v from t_bin where v < 'b' order by v; +id estRows task access object operator info +IndexReader 3323.33 root index:IndexRangeScan +└─IndexRangeScan 3323.33 cop[tikv] table:t_bin, index:v(v) range:[-inf,"b"), keep order:true, stats:pseudo +select v from t_bin where v < 'b' order by v; +v + +a +explain format="brief" select v from t_bin where v < 'b' and v > ' ' order by v; +id estRows task access object operator info +IndexReader 250.00 root index:IndexRangeScan +└─IndexRangeScan 250.00 cop[tikv] table:t_bin, index:v(v) range:(" ","b"), keep order:true, stats:pseudo +select v from t_bin where v < 'b' and v > ' ' order by v; +v +a +explain format="brief" select id from t use index(v) where v < 'b'; +id estRows task access object operator info +Projection 3323.33 root expression__charset_and_collation.t.id +└─IndexLookUp 3323.33 root + ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t, index:v(v) range:[-inf,"\x00B"), keep order:false, stats:pseudo + └─TableRowIDScan(Probe) 3323.33 cop[tikv] table:t keep order:false, stats:pseudo +select id from t use index(v) where v < 'b'; +id +1 +2 +3 +4 +7 +explain format="brief" select id from t use index(v) where v < 'b' and v > ' '; +id estRows task access object operator info +Projection 250.00 root expression__charset_and_collation.t.id +└─IndexLookUp 250.00 root + ├─IndexRangeScan(Build) 250.00 cop[tikv] table:t, index:v(v) range:("","\x00B"), keep order:false, stats:pseudo + └─TableRowIDScan(Probe) 250.00 cop[tikv] table:t keep order:false, stats:pseudo +select id from t use index(v) where v < 'b' and v > ' '; +id +1 +2 +3 +4 +explain format="brief" select id from t_bin use index(v) where v < 'b'; +id estRows task access object operator info +Projection 3323.33 root expression__charset_and_collation.t_bin.id +└─IndexLookUp 3323.33 root + ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t_bin, index:v(v) range:[-inf,"b"), keep order:false, stats:pseudo + └─TableRowIDScan(Probe) 3323.33 cop[tikv] table:t_bin keep order:false, stats:pseudo +select id from t_bin use index(v) where v < 'b'; +id +1 +7 +explain format="brief" select id from t_bin use index(v) where v < 'b' and v > ' '; +id estRows task access object operator info +Projection 250.00 root expression__charset_and_collation.t_bin.id +└─IndexLookUp 250.00 root + ├─IndexRangeScan(Build) 250.00 cop[tikv] table:t_bin, index:v(v) range:(" ","b"), keep order:false, stats:pseudo + └─TableRowIDScan(Probe) 250.00 cop[tikv] table:t_bin keep order:false, stats:pseudo +select id from t_bin use index(v) where v < 'b' and v > ' '; +id +1 +drop table if exists t; +create table t (id int primary key auto_increment, str VARCHAR(20) CHARACTER SET utf8mb4 COLLATE utf8mb4_0900_ai_ci); +insert into t(str) values ('カ'), ('カ'), ('abc'), ('abuFFFEc'), ('abⓒ'), ('𝒶bc'), ('𝕒bc'), ('ガ'), ('が'), ('abç'), ('äbc'), ('ヵ'), ('か'), ('Abc'), ('abC'), ('File-3'), ('file-12'), ('filé-110'), ('🍣'), ('🍺'); +select min(id) from t group by str order by str; +min(id) +19 +20 +3 +4 +18 +17 +16 +1 +select LOCATE('bar' collate utf8mb4_0900_ai_ci, 'FOOBAR' collate utf8mb4_0900_ai_ci); +LOCATE('bar' collate utf8mb4_0900_ai_ci, 'FOOBAR' collate utf8mb4_0900_ai_ci) +4 +select 'FOOBAR' collate utf8mb4_0900_ai_ci REGEXP 'foo.*' collate utf8mb4_0900_ai_ci; +'FOOBAR' collate utf8mb4_0900_ai_ci REGEXP 'foo.*' collate utf8mb4_0900_ai_ci +1 diff --git a/tests/integrationtest/r/expression/issues.result b/tests/integrationtest/r/expression/issues.result new file mode 100644 index 0000000000000..adf6d00292199 --- /dev/null +++ b/tests/integrationtest/r/expression/issues.result @@ -0,0 +1,3167 @@ +drop table if exists t1, t2; +create table t1 (b enum('a', 'b')); +insert into t1 values ('a'); +create table t2 (b enum('b','a') not null, unique(b)); +insert into t2 values ('a'); +select /*+ inl_join(t2)*/ * from t1, t2 where t1.b=t2.b; +b b +a a +drop table if exists t1, t2; +create table t1 (b set('a', 'b')); +insert into t1 values ('a'); +create table t2 (b set('b','a') not null, unique(b)); +insert into t2 values ('a'); +select /*+ inl_join(t2)*/ * from t1, t2 where t1.b=t2.b; +b b +a a +drop table if exists t1, t2; +create table t1 (b enum('a', 'b')); +insert into t1 values ('a'); +create table t2 (b set('b','a') not null, unique(b)); +insert into t2 values ('a'); +select /*+ inl_join(t2)*/ * from t1, t2 where t1.b=t2.b; +b b +a a +drop table if exists t1, t2; +create table t1 (b char(10)); +insert into t1 values ('a'); +create table t2 (b enum('b','a') not null, unique(b)); +insert into t2 values ('a'); +select /*+ inl_join(t2)*/ * from t1, t2 where t1.b=t2.b; +b b +a a +drop table if exists t1, t2; +create table t1 (b char(10)); +insert into t1 values ('a'); +create table t2 (b set('b','a') not null, unique(b)); +insert into t2 values ('a'); +select /*+ inl_join(t2)*/ * from t1, t2 where t1.b=t2.b; +b b +a a +drop table if exists t; +create table t(a decimal(16, 2)); +select sum(case when 1 then a end) from t group by a; +sum(case when 1 then a end) +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` decimal(16,2) DEFAULT NULL +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +drop table if exists t; +create table t(a tinyint, b bit(63)); +insert ignore into t values(599999999, -1); +Level Code Message +Warning 1264 Out of range value for column 'a' at row 1 +Warning 1406 Data too long for column 'b' at row 1 +select hex(a), hex(b) from t; +hex(a) hex(b) +7F 7FFFFFFFFFFFFFFF +drop table if exists t; +create table t(b bit(16)); +insert ignore into t values(0x3635313836),(0x333830); +Level Code Message +Warning 1406 Data too long for column 'b' at row 1 +Warning 1406 Data too long for column 'b' at row 2 +select hex(b) from t; +hex(b) +FFFF +FFFF +drop table if exists t; +create table t(a char(32) not null, b float default '0') engine=innodb default charset=utf8mb4; +insert into t value('0a6f9d012f98467f8e671e9870044528', 208.867); +select concat_ws( ',', b) from t where a = '0a6f9d012f98467f8e671e9870044528'; +concat_ws( ',', b) +208.867 +drop table if exists t; +CREATE TABLE t (b BIT(8)); +INSERT INTO t SET b = b'11111111'; +INSERT INTO t SET b = b'1010'; +INSERT INTO t SET b = b'0101'; +SELECT b+0, BIN(b), OCT(b), HEX(b) FROM t; +b+0 BIN(b) OCT(b) HEX(b) +255 11111111 377 FF +10 1010 12 A +5 101 5 5 +drop table if exists t; +create table t(a char(20)); +select convert(a using a) from t; +Error 1115 (42000): Unknown character set: 'a' +drop table if exists t1; +create table t1(c1 varbinary(100)); +insert into t1 values('abc'); +select 1 from t1 where char_length(c1) = 10; +1 +drop table if exists t; +create table t (id int NOT NULL DEFAULT 8); +SET sql_mode = ''; +insert into t values (1), (NULL), (2); +Level Code Message +Warning 1048 Column 'id' cannot be null +select * from t; +id +1 +0 +2 +set @@sql_mode=default; +drop table if exists t; +create table t(a json); +insert into t values ('"a"'), ('"B"'), ('"c"'), ('"D"'), ('{"a": 1}'), ('1'), ('{"b": 2}'), ('[1, 2]'), ('[3, 4]'); +select min(a) from t; +min(a) +1 +select max(a) from t; +max(a) +[3, 4] +select min(a collate utf8mb4_bin) from t; +min(a collate utf8mb4_bin) +"B" +select max(a collate utf8mb4_bin) from t; +max(a collate utf8mb4_bin) +{"b": 2} +select min(a collate utf8mb4_unicode_ci) from t; +min(a collate utf8mb4_unicode_ci) +"a" +select max(a collate utf8mb4_unicode_ci) from t; +max(a collate utf8mb4_unicode_ci) +1 +SET timestamp=UNIX_TIMESTAMP('2011-11-01 17:48:00'); +SELECT cast(cast('12:12:12' as time) as datetime(6)); +cast(cast('12:12:12' as time) as datetime(6)) +2011-11-01 12:12:12.000000 +drop table if exists t; +create table t (tm time(6)); +insert into t values('23:59:59'); +SELECT cast(tm as datetime(6)) from t; +cast(tm as datetime(6)) +2011-11-01 23:59:59.000000 +drop table if exists `PK_S_MULTI_43`; +CREATE TABLE `PK_S_MULTI_43`(`COL1` time(2) NOT NULL, `COL2` time(2) NOT NULL, `COL3` time(2) DEFAULT NULL, PRIMARY KEY(`COL1`,`COL2`)); +insert into PK_S_MULTI_43(col1, col2) values('-512:37:22.00', '-512:37:22.00'); +select extract(day_microsecond from '-512:37:22.00'); +extract(day_microsecond from '-512:37:22.00') +-5123722000000 +select extract(day_microsecond from col1) from PK_S_MULTI_43; +extract(day_microsecond from col1) +-5123722000000 +drop table if exists t; +create table t(c datetime(6)); +insert into t values('2001-01-01 02:03:04.050607'); +select extract(day_microsecond from cast('2001-01-01 02:03:04.050607' as datetime(6))) from t; +extract(day_microsecond from cast('2001-01-01 02:03:04.050607' as datetime(6))) +1020304050607 +select extract(day_microsecond from c) from t; +extract(day_microsecond from c) +1020304050607 +drop table if exists ft; +create table ft (tint int, tdou double, tdec decimal(22,9),tchar char(44)); +insert into ft values(1234567890,123467890.1234,123467890.1234,'123467890.1234'); +insert into ft values(1234567890,123467890.123456789,123467890.123456789,'123467890.123456789'); +SELECT FROM_UNIXTIME(tchar) from ft; +FROM_UNIXTIME(tchar) +1973-11-30 08:38:10.123400 +1973-11-30 08:38:10.123457 +drop table if exists ft; +create table ft (tint int, tdou double, tdec decimal(22,9),tchar varchar(44)); +insert into ft values(1234567890,123467890.1234,123467890.1234,'123467890.1234'); +insert into ft values(1234567890,123467890.123456789,123467890.123456789,'123467890.123456789'); +SELECT FROM_UNIXTIME(tchar) from ft; +FROM_UNIXTIME(tchar) +1973-11-30 08:38:10.123400 +1973-11-30 08:38:10.123457 +drop table if exists ft; +create table ft (tint int, tdou double, tdec decimal(22,9),tchar blob); +insert into ft values(1234567890,123467890.1234,123467890.1234,'123467890.1234'); +insert into ft values(1234567890,123467890.123456789,123467890.123456789,'123467890.123456789'); +SELECT FROM_UNIXTIME(tchar) from ft; +FROM_UNIXTIME(tchar) +1973-11-30 08:38:10.123400 +1973-11-30 08:38:10.123457 +drop table if exists ft; +create table ft (tint int, tdou double, tdec decimal(22,9),tchar tinyblob); +insert into ft values(1234567890,123467890.1234,123467890.1234,'123467890.1234'); +insert into ft values(1234567890,123467890.123456789,123467890.123456789,'123467890.123456789'); +SELECT FROM_UNIXTIME(tchar) from ft; +FROM_UNIXTIME(tchar) +1973-11-30 08:38:10.123400 +1973-11-30 08:38:10.123457 +drop table if exists ft; +create table ft (tint int, tdou double, tdec decimal(22,9),tchar mediumblob); +insert into ft values(1234567890,123467890.1234,123467890.1234,'123467890.1234'); +insert into ft values(1234567890,123467890.123456789,123467890.123456789,'123467890.123456789'); +SELECT FROM_UNIXTIME(tchar) from ft; +FROM_UNIXTIME(tchar) +1973-11-30 08:38:10.123400 +1973-11-30 08:38:10.123457 +drop table if exists ft; +create table ft (tint int, tdou double, tdec decimal(22,9),tchar longblob); +insert into ft values(1234567890,123467890.1234,123467890.1234,'123467890.1234'); +insert into ft values(1234567890,123467890.123456789,123467890.123456789,'123467890.123456789'); +SELECT FROM_UNIXTIME(tchar) from ft; +FROM_UNIXTIME(tchar) +1973-11-30 08:38:10.123400 +1973-11-30 08:38:10.123457 +truncate table ft; +insert into ft values(1234567890,123467890.1234,123467890.1234,'123467890.1234000000000000000000100111111111'); +SELECT FROM_UNIXTIME(tchar) from ft; +FROM_UNIXTIME(tchar) +1973-11-30 08:38:10.123400 +Level Code Message +Warning 1292 Truncated incorrect DECIMAL value: '123467890.1234000000000000000000100111111111' +truncate table ft; +insert into ft values(1234567890,123467890.1234,123467890.1234,'11111123467890.1234'); +SELECT FROM_UNIXTIME(tchar) from ft; +FROM_UNIXTIME(tchar) +NULL +drop table if exists ft; +create table ft (tint int, tdou double, tdec decimal(22,9),tchar char(44)); +insert into ft values(1234567890,123467890.1234,123467890.1234,'123467890.1234'); +SELECT FROM_UNIXTIME(tchar) from ft where FROM_UNIXTIME(tchar)= '1973-11-30 08:38:10.123400' ; +FROM_UNIXTIME(tchar) +1973-11-30 08:38:10.123400 +SELECT FROM_UNIXTIME(cast(tchar as decimal(44,1))) from ft where FROM_UNIXTIME(tchar)= '1973-11-30 08:38:10.123400' ; +FROM_UNIXTIME(cast(tchar as decimal(44,1))) +1973-11-30 08:38:10.1 +SELECT FROM_UNIXTIME(tchar,'%Y%m%d') from ft where FROM_UNIXTIME(tchar)= '1973-11-30 08:38:10.123400' ; +FROM_UNIXTIME(tchar,'%Y%m%d') +19731130 +drop table if exists t; +create table t(dou double, varc varchar(100)); +insert into t values (1.23e23, '111111111111111111111111111111111111111111111111111111111111111111111111111'); +select from_unixtime(dou) from t; +from_unixtime(dou) +NULL +select from_unixtime(varc) from t; +from_unixtime(varc) +NULL +select from_unixtime(dou, '%Y-%m-%d') from t; +from_unixtime(dou, '%Y-%m-%d') +NULL +select from_unixtime(varc, '%Y-%m-%d') from t; +from_unixtime(varc, '%Y-%m-%d') +NULL +drop table if exists `sun`; +CREATE TABLE `sun` ( `dest` varchar(10) DEFAULT NULL ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin; +insert into sun values('20231020'); +set @@sql_mode = 'ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION'; +set @@tidb_enable_vectorized_expression = on; +select str_to_date(substr(dest,1,6),'%H%i%s') from sun; +str_to_date(substr(dest,1,6),'%H%i%s') +20:23:10 +set @@tidb_enable_vectorized_expression = off; +select str_to_date(substr(dest,1,6),'%H%i%s') from sun; +str_to_date(substr(dest,1,6),'%H%i%s') +20:23:10 +set @@sql_mode=default; +drop table if exists `6bf9e76d-ab44-4031-8a07-418b10741580`, `8919f3f4-25be-4a1a-904a-bb5e863d8fc8`; +CREATE TABLE `6bf9e76d-ab44-4031-8a07-418b10741580` ( +`e0b5f703-6cfe-49b4-bc21-16a6455e43a7` set('7','va','ung60','ow','1g','gxwz5','uhnh','k','5la1','q8d9c','1f') NOT NULL DEFAULT '7,1g,uhnh,5la1,q8d9c', +`fbc3527f-9617-4b9d-a5dc-4be31c00d8a5` datetime DEFAULT '6449-09-28 14:39:04', +PRIMARY KEY (`e0b5f703-6cfe-49b4-bc21-16a6455e43a7`) /*T![clustered_index] CLUSTERED */ +) ENGINE=InnoDB DEFAULT CHARSET=latin1 COLLATE=latin1_bin; +CREATE TABLE `8919f3f4-25be-4a1a-904a-bb5e863d8fc8` ( +`9804d5f2-cbc7-43b7-b241-ea2656dc941a` enum('s951','36d','ua65','49yru','6l2em','4ea','jf2d2','vprsc','3yl7n','hz','ov') DEFAULT '4ea', +`323cdbcb-0c14-4362-90ab-ea42caaed6a5` year(4) NOT NULL DEFAULT '1983', +`b9b70f39-1a02-4114-9d7d-fa6259c1b691` time DEFAULT '20:18:04', +PRIMARY KEY (`323cdbcb-0c14-4362-90ab-ea42caaed6a5`) /*T![clustered_index] CLUSTERED */, +KEY `a704d6bb-772b-44ea-8cb0-6f7491c1aaa6` (`323cdbcb-0c14-4362-90ab-ea42caaed6a5`,`9804d5f2-cbc7-43b7-b241-ea2656dc941a`) +) ENGINE=InnoDB DEFAULT CHARSET=ascii COLLATE=ascii_bin; +delete from `6bf9e76d-ab44-4031-8a07-418b10741580` where not( `6bf9e76d-ab44-4031-8a07-418b10741580`.`e0b5f703-6cfe-49b4-bc21-16a6455e43a7` in ( select `9804d5f2-cbc7-43b7-b241-ea2656dc941a` from `8919f3f4-25be-4a1a-904a-bb5e863d8fc8` where `6bf9e76d-ab44-4031-8a07-418b10741580`.`e0b5f703-6cfe-49b4-bc21-16a6455e43a7` in ( '1f' ) and `6bf9e76d-ab44-4031-8a07-418b10741580`.`e0b5f703-6cfe-49b4-bc21-16a6455e43a7` in ( '1g' ,'va' ,'uhnh' ) ) ) and not( IsNull( `6bf9e76d-ab44-4031-8a07-418b10741580`.`e0b5f703-6cfe-49b4-bc21-16a6455e43a7` ) ); +drop table if exists test; +create table test (name1 blob,name2 blob); +insert into test values(aes_encrypt('a', 'x'), aes_encrypt('b', 'x')); +SELECT * FROM test WHERE CAST(AES_DECRYPT(name1, 'x') AS CHAR) = '00' AND CAST(AES_DECRYPT(name2, 'x') AS CHAR) = '1'; +name1 name2 +drop table if exists ordres; +CREATE TABLE orders (id bigint(20) unsigned NOT NULL ,account_id bigint(20) unsigned NOT NULL DEFAULT '0' ,loan bigint(20) unsigned NOT NULL DEFAULT '0' ,stage_num int(20) unsigned NOT NULL DEFAULT '0' ,apply_time bigint(20) unsigned NOT NULL DEFAULT '0' ,PRIMARY KEY (id) /*T![clustered_index] CLUSTERED */,KEY idx_orders_account_id (account_id),KEY idx_orders_apply_time (apply_time)); +insert into orders values (20, 210802010000721168, 20000 , 2 , 1682484268727), (22, 210802010000721168, 35100 , 4 , 1650885615002); +select min(if(apply_to_now_days <= 30,loan,null)) as min, max(if(apply_to_now_days <= 720,loan,null)) as max from (select loan, datediff(from_unixtime(unix_timestamp('2023-05-18 18:43:43') + 18000), from_unixtime(apply_time/1000 + 18000)) as apply_to_now_days from orders) t1; +min max +20000 35100 +drop table if exists t_tiny, t_small, t_medium, t_int, t_big; +create table t_tiny (c0 TINYINT UNSIGNED); +INSERT IGNORE INTO t_tiny(c0) VALUES (1E9); +Level Code Message +Warning 1264 Out of range value for column 'c0' at row 1 +select * from t_tiny; +c0 +255 +create table t_small (c0 SMALLINT UNSIGNED); +INSERT IGNORE INTO t_small(c0) VALUES (1E9); +Level Code Message +Warning 1264 Out of range value for column 'c0' at row 1 +select * from t_small; +c0 +65535 +create table t_medium (c0 MEDIUMINT UNSIGNED); +INSERT IGNORE INTO t_medium(c0) VALUES (1E9); +Level Code Message +Warning 1264 Out of range value for column 'c0' at row 1 +select * from t_medium; +c0 +16777215 +create table t_int (c0 INT UNSIGNED); +INSERT IGNORE INTO t_int(c0) VALUES (1E20); +Level Code Message +Warning 1264 Out of range value for column 'c0' at row 1 +select * from t_int; +c0 +4294967295 +create table t_big (c0 BIGINT UNSIGNED); +INSERT IGNORE INTO t_big(c0) VALUES (1E20); +Level Code Message +Warning 1264 Out of range value for column 'c0' at row 1 +select * from t_big; +c0 +18446744073709551615 +drop table if exists t; +CREATE TABLE t (a bigint unsigned,key (a)); +INSERT INTO t VALUES (0), (4), (5), (6), (7), (8), (9223372036854775810), (18446744073709551614), (18446744073709551615); +SELECT a FROM t WHERE a NOT IN (-1, -2, 18446744073709551615); +a +0 +4 +5 +6 +7 +8 +9223372036854775810 +18446744073709551614 +SELECT a FROM t WHERE a NOT IN (-1, -2, 4, 9223372036854775810); +a +0 +5 +6 +7 +8 +18446744073709551614 +18446744073709551615 +SELECT a FROM t WHERE a NOT IN (-1, -2, 0, 4, 18446744073709551614); +a +5 +6 +7 +8 +9223372036854775810 +18446744073709551615 +drop table if exists t1; +create table t1 (some_id smallint(5) unsigned,key (some_id) ); +insert into t1 values (1),(2); +select some_id from t1 where some_id not in(2,-1); +some_id +1 +select monthname(str_to_date(null, '%m')), monthname(str_to_date(null, '%m')), monthname(str_to_date(1, '%m')), monthname(str_to_date(0, '%m')); +monthname(str_to_date(null, '%m')) monthname(str_to_date(null, '%m')) monthname(str_to_date(1, '%m')) monthname(str_to_date(0, '%m')) +NULL NULL NULL NULL +select str_to_date(1, '%m'); +str_to_date(1, '%m') +NULL +select str_to_date(01, '%d'); +str_to_date(01, '%d') +NULL +select str_to_date(2019, '%Y'); +str_to_date(2019, '%Y') +NULL +select str_to_date('5,2019','%m,%Y'); +str_to_date('5,2019','%m,%Y') +NULL +select str_to_date('01,2019','%d,%Y'); +str_to_date('01,2019','%d,%Y') +NULL +select str_to_date('01,5','%d,%m'); +str_to_date('01,5','%d,%m') +NULL +set sql_mode='ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION'; +select str_to_date(1, '%m'); +str_to_date(1, '%m') +0000-01-00 +select str_to_date(01, '%d'); +str_to_date(01, '%d') +0000-00-01 +select str_to_date(2019, '%Y'); +str_to_date(2019, '%Y') +2019-00-00 +select str_to_date('5,2019','%m,%Y'); +str_to_date('5,2019','%m,%Y') +2019-05-00 +select str_to_date('01,2019','%d,%Y'); +str_to_date('01,2019','%d,%Y') +2019-00-01 +select str_to_date('01,5','%d,%m'); +str_to_date('01,5','%d,%m') +0000-05-01 +set @@sql_mode=default; +drop table if exists t1, t2; +CREATE TABLE `t1` (`period_name` varchar(24) DEFAULT NULL ,`period_id` bigint(20) DEFAULT NULL ,`starttime` bigint(20) DEFAULT NULL); +CREATE TABLE `t2` (`bussid` bigint(20) DEFAULT NULL,`ct` bigint(20) DEFAULT NULL); +select +a.period_name, +b.date8 +from +(select * from t1) a +left join +(select bussid,date(from_unixtime(ct)) date8 from t2) b +on +a.period_id = b.bussid +where +datediff(b.date8, date(from_unixtime(a.starttime))) >= 0; +period_name date8 +SELECT "1900-01-01 00:00:00" + INTERVAL "100000000:214748364700" MINUTE_SECOND; +"1900-01-01 00:00:00" + INTERVAL "100000000:214748364700" MINUTE_SECOND +8895-03-27 22:11:40 +SELECT "1900-01-01 00:00:00" + INTERVAL 1 << 37 SECOND; +"1900-01-01 00:00:00" + INTERVAL 1 << 37 SECOND +6255-04-08 15:04:32 +SELECT "1900-01-01 00:00:00" + INTERVAL 1 << 31 MINUTE; +"1900-01-01 00:00:00" + INTERVAL 1 << 31 MINUTE +5983-01-24 02:08:00 +SELECT "1900-01-01 00:00:00" + INTERVAL 1 << 38 SECOND; +"1900-01-01 00:00:00" + INTERVAL 1 << 38 SECOND +NULL +SELECT "1900-01-01 00:00:00" + INTERVAL 1 << 33 MINUTE; +"1900-01-01 00:00:00" + INTERVAL 1 << 33 MINUTE +NULL +SELECT "1900-01-01 00:00:00" + INTERVAL 1 << 30 HOUR; +"1900-01-01 00:00:00" + INTERVAL 1 << 30 HOUR +NULL +SELECT "1900-01-01 00:00:00" + INTERVAL "1000000000:214748364700" MINUTE_SECOND; +"1900-01-01 00:00:00" + INTERVAL "1000000000:214748364700" MINUTE_SECOND +NULL +SELECT 19000101000000 + INTERVAL "100000000:214748364700" MINUTE_SECOND; +19000101000000 + INTERVAL "100000000:214748364700" MINUTE_SECOND +8895-03-27 22:11:40 +SELECT 19000101000000 + INTERVAL 1 << 37 SECOND; +19000101000000 + INTERVAL 1 << 37 SECOND +6255-04-08 15:04:32 +SELECT 19000101000000 + INTERVAL 1 << 31 MINUTE; +19000101000000 + INTERVAL 1 << 31 MINUTE +5983-01-24 02:08:00 +SELECT "8895-03-27 22:11:40" - INTERVAL "100000000:214748364700" MINUTE_SECOND; +"8895-03-27 22:11:40" - INTERVAL "100000000:214748364700" MINUTE_SECOND +1900-01-01 00:00:00 +SELECT "6255-04-08 15:04:32" - INTERVAL 1 << 37 SECOND; +"6255-04-08 15:04:32" - INTERVAL 1 << 37 SECOND +1900-01-01 00:00:00 +SELECT "5983-01-24 02:08:00" - INTERVAL 1 << 31 MINUTE; +"5983-01-24 02:08:00" - INTERVAL 1 << 31 MINUTE +1900-01-01 00:00:00 +SELECT "9999-01-01 00:00:00" - INTERVAL 1 << 39 SECOND; +"9999-01-01 00:00:00" - INTERVAL 1 << 39 SECOND +NULL +SELECT "9999-01-01 00:00:00" - INTERVAL 1 << 33 MINUTE; +"9999-01-01 00:00:00" - INTERVAL 1 << 33 MINUTE +NULL +SELECT "9999-01-01 00:00:00" - INTERVAL 1 << 30 HOUR; +"9999-01-01 00:00:00" - INTERVAL 1 << 30 HOUR +NULL +SELECT "9999-01-01 00:00:00" - INTERVAL "10000000000:214748364700" MINUTE_SECOND; +"9999-01-01 00:00:00" - INTERVAL "10000000000:214748364700" MINUTE_SECOND +NULL +SELECT 88950327221140 - INTERVAL "100000000:214748364700" MINUTE_SECOND ; +88950327221140 - INTERVAL "100000000:214748364700" MINUTE_SECOND +1900-01-01 00:00:00 +SELECT 62550408150432 - INTERVAL 1 << 37 SECOND; +62550408150432 - INTERVAL 1 << 37 SECOND +1900-01-01 00:00:00 +SELECT 59830124020800 - INTERVAL 1 << 31 MINUTE; +59830124020800 - INTERVAL 1 << 31 MINUTE +1900-01-01 00:00:00 +SELECT 10000101000000 + INTERVAL "111111111111111111" MICROSECOND; +10000101000000 + INTERVAL "111111111111111111" MICROSECOND +4520-12-21 05:31:51.111111 +SELECT 10000101000000 + INTERVAL "111111111111.111111" SECOND; +10000101000000 + INTERVAL "111111111111.111111" SECOND +4520-12-21 05:31:51.111111 +SELECT 10000101000000 + INTERVAL "111111111111.111111111" SECOND; +10000101000000 + INTERVAL "111111111111.111111111" SECOND +4520-12-21 05:31:51.111111 +SELECT 10000101000000 + INTERVAL "111111111111.111" SECOND; +10000101000000 + INTERVAL "111111111111.111" SECOND +4520-12-21 05:31:51.111000 +SELECT 10000101000000 + INTERVAL "111111111111." SECOND; +10000101000000 + INTERVAL "111111111111." SECOND +4520-12-21 05:31:51 +SELECT 10000101000000 + INTERVAL "111111111111111111.5" MICROSECOND; +10000101000000 + INTERVAL "111111111111111111.5" MICROSECOND +4520-12-21 05:31:51.111112 +SELECT 10000101000000 + INTERVAL "111111111111111112.5" MICROSECOND; +10000101000000 + INTERVAL "111111111111111112.5" MICROSECOND +4520-12-21 05:31:51.111113 +SELECT 10000101000000 + INTERVAL "111111111111111111.500000" MICROSECOND; +10000101000000 + INTERVAL "111111111111111111.500000" MICROSECOND +4520-12-21 05:31:51.111112 +SELECT 10000101000000 + INTERVAL "111111111111111111.50000000" MICROSECOND; +10000101000000 + INTERVAL "111111111111111111.50000000" MICROSECOND +4520-12-21 05:31:51.111112 +SELECT 10000101000000 + INTERVAL "111111111111111111.6" MICROSECOND; +10000101000000 + INTERVAL "111111111111111111.6" MICROSECOND +4520-12-21 05:31:51.111112 +SELECT 10000101000000 + INTERVAL "111111111111111111.499999" MICROSECOND; +10000101000000 + INTERVAL "111111111111111111.499999" MICROSECOND +4520-12-21 05:31:51.111111 +SELECT 10000101000000 + INTERVAL "111111111111111111.499999999999" MICROSECOND; +10000101000000 + INTERVAL "111111111111111111.499999999999" MICROSECOND +4520-12-21 05:31:51.111111 +SELECT INTERVAL 1.123456789e3 SECOND + "1900-01-01 00:00:00"; +INTERVAL 1.123456789e3 SECOND + "1900-01-01 00:00:00" +1900-01-01 00:18:43.456789 +SELECT INTERVAL 1 Year + 19000101000000; +INTERVAL 1 Year + 19000101000000 +1901-01-01 00:00:00 +select interval 6 month + date("1900-01-01"); +interval 6 month + date("1900-01-01") +1900-07-01 +select interval "5:2" MINUTE_SECOND + "1900-01-01"; +interval "5:2" MINUTE_SECOND + "1900-01-01" +1900-01-01 00:05:02 +drop table if exists t; +create table t(a bigint unsigned primary key); +insert into t values(9223372036854775807), (18446744073709551615); +select * from t where a > 9223372036854775807-0.5 order by a; +a +9223372036854775807 +18446744073709551615 +drop table if exists t1, t2; +create table t1(id varchar(36) not null primary key, org_id varchar(36) not null, status tinyint default 1 not null, ns varchar(36) default '' not null); +create table t2(id varchar(36) not null primary key, order_id varchar(36) not null, begin_time timestamp(3) default CURRENT_TIMESTAMP(3) not null); +create index idx_oid on t2(order_id); +insert into t1 value (1,1,1,'a'); +insert into t1 value (2,1,2,'a'); +insert into t1 value (3,1,3,'a'); +insert into t2 value (1,2,date'2020-05-08'); +explain format = 'brief' SELECT /*+ INL_MERGE_JOIN(t1,t2) */ COUNT(*) FROM t1 LEFT JOIN t2 ON t1.id = t2.order_id WHERE t1.ns = 'a' AND t1.org_id IN (1) AND t1.status IN (2,6,10) AND timestampdiff(month, t2.begin_time, date'2020-05-06') = 0; +id estRows task access object operator info +StreamAgg 1.00 root funcs:count(1)->Column#10 +└─IndexMergeJoin 0.03 root inner join, inner:Selection, outer key:expression__issues.t1.id, inner key:expression__issues.t2.order_id + ├─TableReader(Build) 0.02 root data:Selection + │ └─Selection 0.02 cop[tikv] eq(cast(expression__issues.t1.org_id, double BINARY), 1), eq(expression__issues.t1.ns, "a"), in(expression__issues.t1.status, 2, 6, 10) + │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo + └─Selection(Probe) 0.03 root eq(timestampdiff("MONTH", expression__issues.t2.begin_time, 2020-05-06), 0) + └─Projection 0.04 root expression__issues.t2.order_id, expression__issues.t2.begin_time + └─IndexLookUp 0.04 root + ├─IndexRangeScan(Build) 0.04 cop[tikv] table:t2, index:idx_oid(order_id) range: decided by [eq(expression__issues.t2.order_id, expression__issues.t1.id)], keep order:true, stats:pseudo + └─TableRowIDScan(Probe) 0.04 cop[tikv] table:t2 keep order:false, stats:pseudo +SELECT /*+ INL_MERGE_JOIN(t1,t2) */ COUNT(*) FROM t1 LEFT JOIN t2 ON t1.id = t2.order_id WHERE t1.ns = 'a' AND t1.org_id IN (1) AND t1.status IN (2,6,10) AND timestampdiff(month, t2.begin_time, date'2020-05-06') = 0; +COUNT(*) +1 +drop table if exists t; +create table t(a int); +insert into t values(1); +select * from t where a < -184467440737095516167.1; +a +select * from t where a > -184467440737095516167.1; +a +1 +select * from t where a < 184467440737095516167.1; +a +1 +select * from t where a > 184467440737095516167.1; +a +drop table if exists t; +create table t(b bit(1)); +insert into t values(b'1'); +select count(*) from t where b = 1; +count(*) +1 +select count(*) from t where b = '1'; +count(*) +1 +select count(*) from t where b = b'1'; +count(*) +1 +drop table if exists t; +create table t(b bit(63)); +insert into t values(b'111111111111111111111111111111111111111111111111111111111111111'); +select count(*) from t where b = 9223372036854775807; +count(*) +1 +select count(*) from t where b = '9223372036854775807'; +count(*) +1 +select count(*) from t where b = b'111111111111111111111111111111111111111111111111111111111111111'; +count(*) +1 +drop table if exists t1; +CREATE TABLE t1 (v bigint(20) UNSIGNED NOT NULL); +INSERT INTO t1 VALUES (1), (2); +SELECT SUM(IF(v > 1, v, -v)) FROM t1; +SUM(IF(v > 1, v, -v)) +1 +SELECT sum(IFNULL(cast(null+rand() as unsigned), -v)) FROM t1; +sum(IFNULL(cast(null+rand() as unsigned), -v)) +-3 +SELECT sum(COALESCE(cast(null+rand() as unsigned), -v)) FROM t1; +sum(COALESCE(cast(null+rand() as unsigned), -v)) +-3 +SELECT sum(COALESCE(cast(null+rand() as unsigned), v)) FROM t1; +sum(COALESCE(cast(null+rand() as unsigned), v)) +3 +drop table if exists t; +CREATE TABLE t (a decimal(6,3),b double(6,3),c float(6,3)); +INSERT INTO t VALUES (1.100,1.100,1.100); +SELECT DATE_ADD('2003-11-18 07:25:13',INTERVAL a MINUTE_SECOND) FROM t; +DATE_ADD('2003-11-18 07:25:13',INTERVAL a MINUTE_SECOND) +2003-11-18 07:27:53 +SELECT DATE_ADD('2003-11-18 07:25:13',INTERVAL b MINUTE_SECOND) FROM t; +DATE_ADD('2003-11-18 07:25:13',INTERVAL b MINUTE_SECOND) +2003-11-18 07:27:53 +SELECT DATE_ADD('2003-11-18 07:25:13',INTERVAL c MINUTE_SECOND) FROM t; +DATE_ADD('2003-11-18 07:25:13',INTERVAL c MINUTE_SECOND) +2003-11-18 07:27:53 +drop table if exists t; +CREATE TABLE t (a decimal(11,7),b double(11,7),c float(11,7)); +INSERT INTO t VALUES (123.9999999,123.9999999,123.9999999),(-123.9999999,-123.9999999,-123.9999999); +SELECT DATE_ADD('2003-11-18 07:25:13',INTERVAL a MINUTE_SECOND) FROM t; +DATE_ADD('2003-11-18 07:25:13',INTERVAL a MINUTE_SECOND) +2004-03-13 03:14:52 +2003-07-25 11:35:34 +SELECT DATE_ADD('2003-11-18 07:25:13',INTERVAL b MINUTE_SECOND) FROM t; +DATE_ADD('2003-11-18 07:25:13',INTERVAL b MINUTE_SECOND) +2004-03-13 03:14:52 +2003-07-25 11:35:34 +SELECT DATE_ADD('2003-11-18 07:25:13',INTERVAL c MINUTE_SECOND) FROM t; +DATE_ADD('2003-11-18 07:25:13',INTERVAL c MINUTE_SECOND) +2003-11-18 09:29:13 +2003-11-18 05:21:13 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 MINUTE_MICROSECOND); +DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 MINUTE_MICROSECOND) +2007-03-28 22:08:25.800000 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 SECOND_MICROSECOND); +DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 SECOND_MICROSECOND) +2007-03-28 22:08:25.800000 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 HOUR_MICROSECOND); +DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 HOUR_MICROSECOND) +2007-03-28 22:08:25.800000 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 DAY_MICROSECOND); +DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 DAY_MICROSECOND) +2007-03-28 22:08:25.800000 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 SECOND); +DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 SECOND) +2007-03-28 22:08:25.800000 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 HOUR_SECOND); +DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 HOUR_SECOND) +2007-03-28 22:06:26 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 DAY_SECOND); +DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 DAY_SECOND) +2007-03-28 22:06:26 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 MINUTE_SECOND); +DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 MINUTE_SECOND) +2007-03-28 22:06:26 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 MINUTE); +DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 MINUTE) +2007-03-28 22:06:28 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 DAY_MINUTE); +DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 DAY_MINUTE) +2007-03-28 20:06:28 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 HOUR_MINUTE); +DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 HOUR_MINUTE) +2007-03-28 20:06:28 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 DAY_HOUR); +DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 DAY_HOUR) +2007-03-26 20:08:28 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 YEAR_MONTH); +DATE_ADD('2007-03-28 22:08:28',INTERVAL -2.2 YEAR_MONTH) +2005-01-28 22:08:28 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 MINUTE_MICROSECOND); +DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 MINUTE_MICROSECOND) +2007-03-28 22:08:30.200000 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 SECOND_MICROSECOND); +DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 SECOND_MICROSECOND) +2007-03-28 22:08:30.200000 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 HOUR_MICROSECOND); +DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 HOUR_MICROSECOND) +2007-03-28 22:08:30.200000 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 DAY_MICROSECOND); +DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 DAY_MICROSECOND) +2007-03-28 22:08:30.200000 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 SECOND); +DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 SECOND) +2007-03-28 22:08:30.200000 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 HOUR_SECOND); +DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 HOUR_SECOND) +2007-03-28 22:10:30 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 DAY_SECOND); +DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 DAY_SECOND) +2007-03-28 22:10:30 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 MINUTE_SECOND); +DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 MINUTE_SECOND) +2007-03-28 22:10:30 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 MINUTE); +DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 MINUTE) +2007-03-28 22:10:28 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 DAY_MINUTE); +DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 DAY_MINUTE) +2007-03-29 00:10:28 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 HOUR_MINUTE); +DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 HOUR_MINUTE) +2007-03-29 00:10:28 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 DAY_HOUR); +DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 DAY_HOUR) +2007-03-31 00:08:28 +SELECT DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 YEAR_MONTH); +DATE_ADD('2007-03-28 22:08:28',INTERVAL 2.2 YEAR_MONTH) +2009-05-28 22:08:28 +drop table if exists t; +create table t (d decimal(19, 0), i bigint(11)); +insert into t values (123456789012, 123456789012); +select * from t where d = i; +d i +123456789012 123456789012 +select '0-3261554956'+0.0; +'0-3261554956'+0.0 +0 +select cast('0-1234' as real); +cast('0-1234' as real) +0 +drop table if exists testValuesBinary; +create table testValuesBinary(id int primary key auto_increment, a bit(1)); +insert into testValuesBinary values(1,1); +insert into testValuesBinary values(1,1) on duplicate key update id = values(id),a = values(a); +select a=0 from testValuesBinary; +a=0 +0 +insert into testValuesBinary values(1,0) on duplicate key update id = values(id),a = values(a); +select a=0 from testValuesBinary; +a=0 +1 +drop table testValuesBinary; +DROP TABLE IF EXISTS t; +CREATE TABLE t (v VARCHAR(100)); +INSERT INTO t VALUES ('3289742893213123732904809'); +SELECT * FROM t WHERE v; +v +3289742893213123732904809 +drop table if exists tt; +create table tt(a varchar(10)); +insert into tt values(NULL); +analyze table tt; +select * from tt; +a +NULL +select collation(format_bytes(1024)) != 'binary'; +collation(format_bytes(1024)) != 'binary' +1 +select collation(format_nano_time(234)) != 'binary'; +collation(format_nano_time(234)) != 'binary' +1 +drop table if exists t; +create table t (a char(10) charset gbk collate gbk_chinese_ci, b time); +insert into t values ('08:00:00', '08:00:00'); +select t1.a, t2.b from t as t1 right join t as t2 on t1.a = t2.b; +a b +08:00:00 08:00:00 +drop table if exists table_30_utf8_4, t; +create table t(a int); +insert into t values(1); +create table table_30_utf8_4 ( `pk` int primary key, `col_int_key_unsigned` int unsigned , `col_int_key_signed` int, `col_float_key_signed` float , `col_float_key_unsigned` float unsigned) character set utf8 partition by hash(pk) partitions 4; +insert ignore into table_30_utf8_4 values (0,91, 10, 14,19.0495); +alter table table_30_utf8_4 add column a int as (col_int_key_signed * 2); +SELECT count(1) AS val FROM table_30_utf8_4 WHERE table_30_utf8_4.col_int_key_unsigned!=table_30_utf8_4.a OR (SELECT count(1) AS val FROM t WHERE table_30_utf8_4.col_float_key_signed!=table_30_utf8_4.col_float_key_unsigned )!=7984764426240273913; +val +1 +select a from table_30_utf8_4 order by a; +a +20 +drop table if exists t1; +CREATE TABLE t1 ( id INT NOT NULL PRIMARY KEY auto_increment, pad VARCHAR(10) NOT NULL, expr varchar(100) AS (NOT 1 BETWEEN -5 AND 5)); +INSERT INTO t1 (pad) VALUES ('a'), ('b'); +SELECT id, pad, expr, NOT 1 BETWEEN -5 AND 5 as expr_in_select FROM t1; +id pad expr expr_in_select +1 a 0 0 +2 b 0 0 +drop table if exists t0; +CREATE TABLE t0(c0 int); +INSERT INTO t0 VALUES (0); +SELECT t0.c0 FROM t0 WHERE CHAR(204355900); +c0 +0 +SELECT t0.c0 FROM t0 WHERE not CHAR(204355900); +c0 +SELECT t0.c0 FROM t0 WHERE '.0'; +c0 +SELECT t0.c0 FROM t0 WHERE not '.0'; +c0 +0 +select * from t0 where '.00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000009'; +c0 +0 +select * from t0 where not '.00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000009'; +c0 +select * from t0 where '.000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000009'; +c0 +select * from t0 where not '.000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000009'; +c0 +0 +drop table if exists testjson; +CREATE TABLE testjson (j json DEFAULT NULL) ENGINE=InnoDB DEFAULT CHARSET=utf8; +INSERT INTO testjson SET j='{"test":3}'; +INSERT INTO testjson SET j='{"test":0}'; +insert into testjson set j='{"test":"0"}'; +insert into testjson set j='{"test":0.0}'; +INSERT INTO testjson SET j='{"test":"aaabbb"}'; +INSERT INTO testjson SET j='{"test":3.1415}'; +INSERT INTO testjson SET j='{"test":[]}'; +INSERT INTO testjson SET j='{"test":[1,2]}'; +INSERT INTO testjson SET j='{"test":["b","c"]}'; +INSERT INTO testjson SET j='{"test":{"ke":"val"}}'; +insert into testjson set j='{"test":"2015-07-27 09:43:47"}'; +insert into testjson set j='{"test":"0000-00-00 00:00:00"}'; +insert into testjson set j='{"test":"0778"}'; +insert into testjson set j='{"test":"0000"}'; +insert into testjson set j='{"test":null}'; +insert into testjson set j=null; +insert into testjson set j='{"test":[null]}'; +insert into testjson set j='{"test":true}'; +insert into testjson set j='{"test":false}'; +insert into testjson set j='""'; +insert into testjson set j='null'; +insert into testjson set j='0'; +insert into testjson set j='"0"'; +SELECT * FROM testjson WHERE JSON_EXTRACT(j,'$.test'); +j +{"test": 3} +{"test": "0"} +{"test": "aaabbb"} +{"test": 3.1415} +{"test": []} +{"test": [1, 2]} +{"test": ["b", "c"]} +{"test": {"ke": "val"}} +{"test": "2015-07-27 09:43:47"} +{"test": "0000-00-00 00:00:00"} +{"test": "0778"} +{"test": "0000"} +{"test": null} +{"test": [null]} +{"test": true} +{"test": false} +select * from testjson where j; +j +{"test": 3} +{"test": 0} +{"test": "0"} +{"test": 0} +{"test": "aaabbb"} +{"test": 3.1415} +{"test": []} +{"test": [1, 2]} +{"test": ["b", "c"]} +{"test": {"ke": "val"}} +{"test": "2015-07-27 09:43:47"} +{"test": "0000-00-00 00:00:00"} +{"test": "0778"} +{"test": "0000"} +{"test": null} +{"test": [null]} +{"test": true} +{"test": false} +"" +null +"0" +insert into mysql.expr_pushdown_blacklist values('json_extract','tikv',''); +admin reload expr_pushdown_blacklist; +SELECT * FROM testjson WHERE JSON_EXTRACT(j,'$.test'); +j +{"test": 3} +{"test": "0"} +{"test": "aaabbb"} +{"test": 3.1415} +{"test": []} +{"test": [1, 2]} +{"test": ["b", "c"]} +{"test": {"ke": "val"}} +{"test": "2015-07-27 09:43:47"} +{"test": "0000-00-00 00:00:00"} +{"test": "0778"} +{"test": "0000"} +{"test": null} +{"test": [null]} +{"test": true} +{"test": false} +select * from testjson where j; +j +{"test": 3} +{"test": 0} +{"test": "0"} +{"test": 0} +{"test": "aaabbb"} +{"test": 3.1415} +{"test": []} +{"test": [1, 2]} +{"test": ["b", "c"]} +{"test": {"ke": "val"}} +{"test": "2015-07-27 09:43:47"} +{"test": "0000-00-00 00:00:00"} +{"test": "0778"} +{"test": "0000"} +{"test": null} +{"test": [null]} +{"test": true} +{"test": false} +"" +null +"0" +delete from mysql.expr_pushdown_blacklist; +admin reload expr_pushdown_blacklist; +drop table if exists t0; +CREATE TABLE t0(c0 int); +INSERT INTO t0 VALUES (1); +SELECT * FROM t0 WHERE 1 AND 0.4; +c0 +1 +drop table if exists t; +create table t(a int); +insert into t values(2); +select * from t where (not not a) = a; +a +select * from t where (not not not not a) = a; +a +drop table if exists t0; +CREATE TABLE t0(c0 INT); +INSERT INTO t0(c0) VALUES (0); +SELECT * FROM t0 WHERE -10000000000000000000 | t0.c0 UNION SELECT * FROM t0; +c0 +0 +SELECT * FROM t0 WHERE -10000000000000000000 | t0.c0 UNION all SELECT * FROM t0; +c0 +0 +0 +drop table if exists t0; +CREATE TABLE t0(c0 TEXT(10)); +INSERT INTO t0(c0) VALUES (1); +SELECT * FROM t0 WHERE ('a' != t0.c0) AND t0.c0; +c0 +1 +CREATE INDEX i0 ON t0(c0(10)); +SELECT * FROM t0 WHERE ('a' != t0.c0) AND t0.c0; +c0 +1 +drop table if exists t0; +CREATE TABLE t0(c0 INT, c1 INT AS (c0)); +CREATE INDEX i0 ON t0(c1); +SELECT t0.c0 FROM t0 UNION ALL SELECT 0 FROM t0; +c0 +drop table if exists t0, t1; +CREATE TABLE t0(c0 INT); +CREATE TABLE t1(c0 INT); +SELECT * FROM t1 NATURAL LEFT JOIN t0 WHERE NOT t1.c0; +c0 +drop table if exists t0,t1; +CREATE TABLE t0(c0 INT); +CREATE TABLE t1(c0 INT); +INSERT INTO t0 VALUES (NULL), (1); +INSERT INTO t1 VALUES (0); +SELECT t0.c0 FROM t0 JOIN t1 ON (t0.c0 REGEXP 1) | t1.c0 WHERE BINARY STRCMP(t1.c0, t0.c0); +c0 +1 +drop table if exists t; +create table t (a int); +insert into t values (42); +select a from t where a/10000; +a +42 +select a from t where a/100000; +a +42 +select a from t where a/1000000; +a +42 +select a from t where a/10000000; +a +42 +drop table if exists t; +create table t(a datetime, b year); +insert into t values('2000-05-03 16:44:44', 2018); +insert into t values('2020-10-01 11:11:11', 2000); +insert into t values('2020-10-01 11:11:11', 2070); +insert into t values('2020-10-01 11:11:11', 1999); +select * from t where t.a < t.b; +a b +2000-05-03 16:44:44 2018 +2020-10-01 11:11:11 2070 +select * from t where t.a > t.b; +a b +2020-10-01 11:11:11 2000 +2020-10-01 11:11:11 1999 +drop table if exists tt; +create table tt(a date, b year); +insert into tt values('2019-11-11', 2000); +insert into tt values('2019-11-11', 2020); +insert into tt values('2019-11-11', 2022); +select * from tt where tt.a > tt.b; +a b +2019-11-11 2000 +select * from tt where tt.a < tt.b; +a b +2019-11-11 2020 +2019-11-11 2022 +drop table if exists ttt; +create table ttt(a timestamp, b year); +insert into ttt values('2019-11-11 11:11:11', 2019); +insert into ttt values('2019-11-11 11:11:11', 2000); +insert into ttt values('2019-11-11 11:11:11', 2022); +select * from ttt where ttt.a > ttt.b; +a b +2019-11-11 11:11:11 2019 +2019-11-11 11:11:11 2000 +select * from ttt where ttt.a < ttt.b; +a b +2019-11-11 11:11:11 2022 +drop table if exists t0, t1; +create table t0 (c0 int); +create table t1 (c0 int); +SELECT * FROM t1 LEFT JOIN t0 ON TRUE WHERE BINARY EXPORT_SET(0, 0, 0 COLLATE 'binary', t0.c0, 0 COLLATE 'binary'); +c0 c0 +drop table if exists t; +create table t(a int,b varchar(20),c datetime,d double,e int,f int as(a+b),key(a),key(b),key(c),key(d),key(e),key(f)); +insert into t(a,b,e) values(null,"5",null); +insert into t(a,b,e) values("5",null,null); +select /*+ use_index_merge(t)*/ * from t where t.e=5 or t.a=5; +a b c d e f +5 NULL NULL NULL NULL NULL +drop table if exists t1, t2; +create table t1(a char) collate utf8mb4_bin; +create table t2(a char) collate utf8mb4_bin; +insert into t1 values('a'); +insert into t2 values('a'); +select collation(t1.a) from t1 union select collation(t2.a) from t2; +collation(t1.a) +utf8mb4_bin +select collation(user()); +collation(user()) +utf8mb4_bin +select collation(compress('abc')); +collation(compress('abc')) +binary +set tidb_enable_prepared_plan_cache=1; +drop table if exists t; +set @@tidb_enable_vectorized_expression = false; +create table t(a datetime); +insert into t values(from_unixtime(1589873945)), (from_unixtime(1589873946)); +prepare stmt7 from 'SELECT unix_timestamp(a) FROM t WHERE a = from_unixtime(?);'; +set @val1 = 1589873945; +set @val2 = 1589873946; +execute stmt7 using @val1; +unix_timestamp(a) +1589873945 +execute stmt7 using @val2; +unix_timestamp(a) +1589873946 +drop table if exists t0; +create table t0(a char(10), b int as ((a))); +insert into t0(a) values("0.5"); +select * from t0; +a b +0.5 1 +drop table if exists t; +create table t(a int, b json, c int AS (JSON_EXTRACT(b, '$.population')), key(c)); +select /*+ TIDB_INLJ(t2) */ t1.a, t1.c, t2.a from t t1, t t2 where t1.c=t2.c; +a c a +drop table if exists t; +CREATE TABLE t (id int(10) unsigned NOT NULL AUTO_INCREMENT,type tinyint(4) NOT NULL,create_time int(11) NOT NULL,PRIMARY KEY (id)); +insert into t values (4, 2, 1598584933); +select from_unixtime(create_time,'%Y-%m-%d') as t_day,count(*) as cnt from t where `type` = 1 group by t_day union all select from_unixtime(create_time,'%Y-%m-%d') as t_day,count(*) as cnt from t where `type` = 2 group by t_day; +t_day cnt +2020-08-28 1 +drop table if exists t1; +create table t1 (col0 BLOB, col1 CHAR(74), col2 DATE UNIQUE); +insert into t1 values ('l', '7a34bc7d-6786-461b-92d3-fd0a6cd88f39', '1000-01-03'); +insert into t1 values ('l', NULL, '1000-01-04'); +insert into t1 values ('b', NULL, '1000-01-02'); +select INTERVAL( ( CONVERT( -11752 USING utf8 ) ), 6558853612195285496, `col1`) from t1; +INTERVAL( ( CONVERT( -11752 USING utf8 ) ), 6558853612195285496, `col1`) +0 +0 +0 +drop table if exists t, t1; +create table t(a int, b enum('A', 'B')); +create table t1(a1 int, b1 enum('B', 'A')); +insert into t values (1, 'A'); +insert into t1 values (1, 'A'); +select /*+ HASH_JOIN(t, t1) */ * from t join t1 on t.b = t1.b1; +a b a1 b1 +1 A 1 A +drop table t, t1; +create table t(a int, b set('A', 'B')); +create table t1(a1 int, b1 set('B', 'A')); +insert into t values (1, 'A'); +insert into t1 values (1, 'A'); +select /*+ HASH_JOIN(t, t1) */ * from t join t1 on t.b = t1.b1; +a b a1 b1 +1 A 1 A +drop table if exists t1; +create table t1 (c_int int, primary key (c_int)); +insert into t1 values (1), (2), (3); +drop table if exists t2; +create table t2 (c_int int, primary key (c_int)); +insert into t2 values (1); +select (select count(c_int) from t2 where c_int = t1.c_int) c1, (select count(1) from t2 where c_int = t1.c_int) c2 from t1; +c1 c2 +1 1 +0 0 +0 0 +select (select count(c_int*c_int) from t2 where c_int = t1.c_int) c1, (select count(1) from t2 where c_int = t1.c_int) c2 from t1; +c1 c2 +1 1 +0 0 +0 0 +drop table if exists t0; +CREATE TABLE t0(c0 INTEGER AS (NULL) NOT NULL, c1 INT); +CREATE INDEX i0 ON t0(c0, c1); +INSERT IGNORE INTO t0(c1) VALUES (0); +SELECT * FROM t0; +c0 c1 +0 0 +begin; +INSERT IGNORE INTO t0(c1) VALUES (0); +SELECT * FROM t0; +c0 c1 +0 0 +0 0 +rollback; +drop table if exists t; +create table t (a int) partition by range(a) (PARTITION p0 VALUES LESS THAN (10)); +alter table t add partition (partition p1 values less than (a)); +Error 1054 (42S22): Unknown column 'a' in 'expression' +select * from t; +a +drop table if exists t; +create table t (a int) partition by range(a) (PARTITION p0 VALUES LESS THAN (a)); +Error 1054 (42S22): Unknown column 'a' in 'expression' +DROP TABLE IF EXISTS `table_float`; +DROP TABLE IF EXISTS `table_int_float_varchar`; +CREATE TABLE `table_float` (`id_1` int(16) NOT NULL AUTO_INCREMENT,`col_float_1` float DEFAULT NULL,PRIMARY KEY (`id_1`)) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin AUTO_INCREMENT=97635; +CREATE TABLE `table_int_float_varchar` (`id_6` int(16) NOT NULL AUTO_INCREMENT,`col_int_6` int(16) DEFAULT NULL,`col_float_6` float DEFAULT NULL,`col_varchar_6` varchar(511) DEFAULT NULL,PRIMARY KEY (`id_6`),KEY `vhyen` (`id_6`,`col_int_6`,`col_float_6`,`col_varchar_6`(1)),KEY `zzylq` (`id_6`,`col_int_6`,`col_float_6`,`col_varchar_6`(1))) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin AUTO_INCREMENT=90818; +INSERT INTO `table_float` VALUES (1,NULL),(2,0.1),(3,0),(4,-0.1),(5,-0.1),(6,NULL),(7,0.5),(8,0),(9,0),(10,NULL),(11,1),(12,1.5),(13,NULL),(14,NULL); +INSERT INTO `table_int_float_varchar` VALUES (1,0,0.1,'true'),(2,-1,1.5,'2020-02-02 02:02:00'),(3,NULL,1.5,NULL),(4,65535,0.1,'true'),(5,NULL,0.1,'1'),(6,-1,1.5,'2020-02-02 02:02:00'),(7,-1,NULL,''),(8,NULL,-0.1,NULL),(9,NULL,-0.1,'1'),(10,-1,NULL,''),(11,NULL,1.5,'false'),(12,-1,0,NULL),(13,0,-0.1,NULL),(14,-1,NULL,'-0'),(15,65535,-1,'1'),(16,NULL,0.5,NULL),(17,-1,NULL,NULL); +select count(*) from table_float +JOIN table_int_float_varchar AS tmp3 ON (tmp3.col_varchar_6 AND NULL) +IS NULL WHERE col_int_6=0; +count(*) +14 +SELECT count(*) FROM (table_float JOIN table_int_float_varchar AS tmp3 ON (tmp3.col_varchar_6 AND NULL) IS NULL); +count(*) +154 +SELECT * FROM (table_int_float_varchar AS tmp3) WHERE (col_varchar_6 AND NULL) IS NULL AND col_int_6=0; +id_6 col_int_6 col_float_6 col_varchar_6 +13 0 -0.1 NULL +drop table if exists papers; +create table papers(title text, content longtext); +insert into papers values('title', 'content'); +select to_base64(title), to_base64(content) from papers; +to_base64(title) to_base64(content) +dGl0bGU= Y29udGVudA== +set tidb_enable_vectorized_expression = 0; +select to_base64(title), to_base64(content) from papers; +to_base64(title) to_base64(content) +dGl0bGU= Y29udGVudA== +set tidb_enable_vectorized_expression = 1; +drop table if exists t, t1; +create table t(a enum('a', 'b'), b tinyint); +create table t1(c varchar(20)); +insert into t values('b', 0); +insert into t1 values('b'); +select * from t, t1 where t.a= t1.c; +a b c +b 0 b +select * from t, t1 where t.b= t1.c; +a b c +b 0 b +select * from t, t1 where t.a = t1.c and t.b= t1.c; +a b c +b 0 b +drop table if exists t; +create table t(a enum('a','b')); +insert into t values('b'); +select * from t where a > 1 and a = "b"; +a +b +drop table if exists lt; +create table lt (d decimal(10, 4)); +insert into lt values(0.2),(0.2); +select LEAD(d,1,1) OVER(), LAG(d,1,1) OVER() from lt; +LEAD(d,1,1) OVER() LAG(d,1,1) OVER() +0.2000 1.0000 +1.0000 0.2000 +drop table if exists t; +create table t(a int); +insert into t values (1); +insert into t select values(a) from t; +select * from t; +a +1 +NULL +DROP TABLE IF EXISTS tmp; +CREATE TABLE tmp (id int(11) NOT NULL,value int(1) NOT NULL,PRIMARY KEY (id)); +INSERT INTO tmp VALUES (1, 1),(2,2),(3,3),(4,4),(5,5); +SET @sum := 10; +SELECT @sum := IF(@sum=20,4,@sum + tmp.value) sum FROM tmp ORDER BY tmp.id; +sum +11 +13 +16 +20 +4 +drop table if exists t; +create table t(id int primary key, c int, d timestamp null default null); +insert into t values(1, 2, '2038-01-18 20:20:30'); +update t set d = adddate(d, interval 1 day) where id < 10; +Error 1292 (22007): Incorrect timestamp value: '2038-01-19 20:20:30' +drop view if exists t15847; +CREATE VIEW t15847(c0) AS SELECT NULL; +SELECT * FROM t15847 WHERE (NOT (IF(t15847.c0, NULL, NULL))); +c0 +drop view if exists t15847; +select json_array(true); +json_array(true) +[true] +select json_array(1=2); +json_array(1=2) +[false] +select json_array(1!=2); +json_array(1!=2) +[true] +select json_array(1<2); +json_array(1<2) +[true] +select json_array(1<=2); +json_array(1<=2) +[true] +select json_array(1>2); +json_array(1>2) +[false] +select json_array(1>=2); +json_array(1>=2) +[false] +select json_object(true, null <=> null); +json_object(true, null <=> null) +{"1": true} +select json_object(false, 1 and 2); +json_object(false, 1 and 2) +{"0": true} +select json_object(false, 1 and 0); +json_object(false, 1 and 0) +{"0": false} +select json_object(false, 1 or 0); +json_object(false, 1 or 0) +{"0": true} +select json_object(false, 1 xor 0); +json_object(false, 1 xor 0) +{"0": true} +select json_object(false, 1 xor 1); +json_object(false, 1 xor 1) +{"0": false} +select json_object(false, not 1); +json_object(false, not 1) +{"0": false} +select json_array(null and 1); +json_array(null and 1) +[null] +select json_array(null and 0); +json_array(null and 0) +[false] +select json_array(null or 1); +json_array(null or 1) +[true] +select json_array(null or 0); +json_array(null or 0) +[null] +select json_array(1.15 or 0); +json_array(1.15 or 0) +[true] +select json_array('abc' or 0); +json_array('abc' or 0) +[false] +select json_array('1abc' or 0); +json_array('1abc' or 0) +[true] +select json_array(null is true); +json_array(null is true) +[false] +select json_array(null is null); +json_array(null is null) +[true] +select json_array(1 in (1, 2)); +json_array(1 in (1, 2)) +[true] +select json_array(0 in (1, 2)); +json_array(0 in (1, 2)) +[false] +select json_array(0 not in (1, 2)); +json_array(0 not in (1, 2)) +[true] +select json_array(1 between 0 and 2); +json_array(1 between 0 and 2) +[true] +select json_array(1 not between 0 and 2); +json_array(1 not between 0 and 2) +[false] +select json_array('123' like '123'); +json_array('123' like '123') +[true] +select json_array('abcdef' rlike 'a.*c.*'); +json_array('abcdef' rlike 'a.*c.*') +[true] +select json_array(is_ipv4('127.0.0.1')); +json_array(is_ipv4('127.0.0.1')) +[true] +select json_array(is_ipv6('1a6b:8888:ff66:77ee:0000:1234:5678:bcde')); +json_array(is_ipv6('1a6b:8888:ff66:77ee:0000:1234:5678:bcde')) +[true] +drop table if exists t7; +create table t7 (col0 SMALLINT, col1 VARBINARY(1), col2 DATE, col3 BIGINT, col4 BINARY(166)); +insert into t7 values ('32767', '', '1000-01-03', '-0', '11101011'); +select col2 = 1 from t7; +col2 = 1 +0 +select col2 != 1 from t7; +col2 != 1 +1 +select CAST("9223372036854775808" as json); +CAST("9223372036854775808" as json) +9223372036854775808 +select json_type(CAST("9223372036854775808" as json)); +json_type(CAST("9223372036854775808" as json)) +UNSIGNED INTEGER +select CAST(9223372036854775808 as json); +CAST(9223372036854775808 as json) +9223372036854775808 +select json_type(CAST(9223372036854775808 as json)); +json_type(CAST(9223372036854775808 as json)) +UNSIGNED INTEGER +select CAST(-9223372036854775808 as json); +CAST(-9223372036854775808 as json) +-9223372036854775808 +select json_type(CAST(-9223372036854775808 as json)); +json_type(CAST(-9223372036854775808 as json)) +INTEGER +drop table if exists tx2; +create table tx2 (col json); +insert into tx2 values (json_array("3")),(json_array("3")),(json_array("3")),(json_array("3")); +insert into tx2 values (json_array(3.0)); +insert into tx2 values (json_array(3)); +insert into tx2 values (json_array(3.0)); +insert into tx2 values (json_array(-3)); +insert into tx2 values (json_array(-3.0)); +insert into tx2 values (json_array(922337203685477580)); +insert into tx2 values (json_array(922337203685477581)),(json_array(922337203685477581)),(json_array(922337203685477581)),(json_array(922337203685477581)),(json_array(922337203685477581)); +select col, count(1) c from tx2 group by col order by c desc; +col c +[922337203685477581] 5 +["3"] 4 +[3] 3 +[-3] 2 +[922337203685477580] 1 +drop table if exists t12205; +create table t12205( +`col_varchar_64` varchar(64) DEFAULT NULL, +`col_varchar_64_key` varchar(64) DEFAULT NULL +); +insert into t12205 values('-1038024704','-527892480'); +select SEC_TO_TIME( ( `col_varchar_64` & `col_varchar_64_key` ) ),`col_varchar_64` & `col_varchar_64_key` from t12205; +SEC_TO_TIME( ( `col_varchar_64` & `col_varchar_64_key` ) ) `col_varchar_64` & `col_varchar_64_key` +838:59:59 18446744072635875328 +Level Code Message +Warning 1292 Truncated incorrect time value: '18446744072635875000' +drop table if exists t; +create table t(1e int); +insert into t values (1); +select t.1e from expression__issues.t; +1e +1 +drop table if exists t; +create table t(99e int, r10 int); +insert into t values (1, 10), (2, 2); +select 99e+r10 from t; +99e+r10 +11 +4 +select .78$123; +$123 +0.78 +select .78$421+1; +Error 1064 (42000): You have an error in your SQL syntax; check the manual that corresponds to your TiDB version for the right syntax to use line 1 column 15 near "+1;" +select t. `r10` > 3 from t; +t. `r10` > 3 +1 +0 +select * from t where t. `r10` > 3; +99e r10 +1 10 +drop table if exists t1; +create table t1 (f1 decimal(5,5)); +insert into t1 values (-0.12345); +select concat(f1) from t1; +concat(f1) +-0.12345 +drop table if exists t12206; +create table t12206( +`col_tinyint_unsigned` tinyint(3) unsigned DEFAULT NULL, +`col_double_unsigned` double unsigned DEFAULT NULL, +`col_year_key` year(4) DEFAULT NULL +); +insert into t12206 values(73,0,0000); +SELECT TIME_FORMAT( `col_tinyint_unsigned`, ( IFNULL( `col_double_unsigned`, `col_year_key` ) ) ) AS field1 FROM `t12206`; +field1 +NULL +Level Code Message +Warning 1292 Truncated incorrect time value: '73' +drop table if exists t12209; +create table t12209(a bigint(20)); +insert into t12209 values(1); +select `a` DIV ( ROUND( ( SCHEMA() ), '1978-05-18 03:35:52.043591' ) ) from `t12209`; +`a` DIV ( ROUND( ( SCHEMA() ), '1978-05-18 03:35:52.043591' ) ) +NULL +drop table if exists ta, tb; +CREATE TABLE `ta` ( `k` varchar(32) NOT NULL DEFAULT ' ', `c0` varchar(32) NOT NULL DEFAULT ' ', `c` varchar(18) NOT NULL DEFAULT ' ', `e0` varchar(1) NOT NULL DEFAULT ' ', PRIMARY KEY (`k`,`c0`,`c`), KEY `idx` (`c`,`e0`)) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin; +CREATE TABLE `tb` ( `k` varchar(32) NOT NULL DEFAULT ' ', `e` int(11) NOT NULL DEFAULT '0', `i` int(11) NOT NULL DEFAULT '0', `s` varchar(1) NOT NULL DEFAULT ' ', `c` varchar(50) NOT NULL DEFAULT ' ', PRIMARY KEY (`k`)) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin; +prepare stmt from "select a.* from ta a left join tb b on a.k = b.k where (a.k <> '000000' and ((b.s = ? and i = ? ) or (b.s = ? and e = ?) or (b.s not in(?, ?))) and b.c like '%1%') or (a.c <> '000000' and a.k = '000000')"; +set @a=3;set @b=20200414;set @c='a';set @d=20200414;set @e=3;set @f='a'; +execute stmt using @a,@b,@c,@d,@e,@f; +k c0 c e0 +drop table if exists t; +create table t( +a enum('a','','c'), +b enum('0','1','2'), +c set('a','','c'), +d set('0','1','2') +); +insert into t values(1,1,1,1),(2,2,2,2),(3,3,3,3); +set @@sql_mode = ''; +insert into t values('','','',''); +select * from t; +a b c d +a 0 a 0 + 1 1 +c 2 a, 0,1 + +select a from t where a; +a +a + +c + +select b from t where b; +b +0 +1 +2 +select c from t where c; +c +a + +a, + +select d from t where d; +d +0 +1 +0,1 +set @@sql_mode=default; +drop table if exists t; +create table t(a year); +insert into t values(2002); +select * from t where a=2; +a +2002 +select * from t where a='2'; +a +2002 +drop table if exists t1, t2; +create table t1 (c_int int, c_str varchar(40), primary key (c_int, c_str) clustered, key(c_int) ); +create table t2 like t1 ; +insert into t1 values (1, 'crazy lumiere'), (10, 'goofy mestorf'); +insert into t2 select * from t1 ; +select (select t2.c_str from t2 where t2.c_str = t1.c_str and t2.c_int = 10 order by t2.c_str limit 1) x from t1; +x +NULL +goofy mestorf +drop table if exists t1; +create table t1(c1 int); +insert into t1 values(-2147483648), (-2147483648), (null); +select count(*) from t1 where c1 > (select sum(c1) from t1); +count(*) +2 +drop table if exists t; +create table t(a int primary key, b set('Alice','Bob') DEFAULT NULL); +insert into t value(1,'Bob'); +select max(b) + 0 from t group by a; +max(b) + 0 +2 +drop table if exists t; +create table t(a int, b set('Alice','Bob') DEFAULT NULL); +insert into t value(1,'Bob'); +select max(b) + 0 from t group by a; +max(b) + 0 +2 +drop table if exists test_decimal,test_t; +create table test_decimal(col_decimal decimal(10,0)); +insert into test_decimal values(null),(8); +create table test_t(a int(11), b decimal(32,0)); +insert into test_t values(1,4),(2,4),(5,4),(7,4),(9,4); +SELECT ( test_decimal . `col_decimal` , test_decimal . `col_decimal` ) IN ( select * from test_t ) as field1 FROM test_decimal; +field1 +NULL +0 +drop table if exists foo, bar; +create table foo(a decimal(65,0)); +create table bar(a decimal(65,0), b decimal(65,0)); +insert into bar values(0,0),(1,1),(2,2); +insert into foo select if(b>0, if(a/b>1, 1, 2), null) from bar; +drop table if exists t1_1, t2_1; +CREATE TABLE `t1_1` (`col1` double DEFAULT NULL, `col2` double DEFAULT NULL); +CREATE TABLE `t2_1` (`col1` varchar(20) DEFAULT NULL, `col2` double DEFAULT NULL); +insert into t1_1 values(12.991, null), (12.991, null); +insert into t2_1(col2) values(87), (-9.183), (-9.183); +set @@tidb_enable_vectorized_expression = false; +select t1.col1, t2.col1, t2.col2 from t1_1 t1 inner join t2_1 t2 on t1.col1 not in (1,t2.col1,t2.col2) order by 1,2,3; +col1 col1 col2 +set @@tidb_enable_vectorized_expression = true; +select t1.col1, t2.col1, t2.col2 from t1_1 t1 inner join t2_1 t2 on t1.col1 not in (1,t2.col1,t2.col2) order by 1,2,3; +col1 col1 col2 +drop table if exists tbl_6, tbl_17; +create table tbl_6 (col_31 year, index(col_31)); +create table tbl_17 (col_102 int, col_105 int); +replace into tbl_17 (col_102, col_105) values (9999, 0); +select tbl_6.col_31 from tbl_6 where col_31 in (select col_102 from tbl_17 where tbl_17.col_102 = 9999 and tbl_17.col_105 = 0); +col_31 +drop table if exists tbl_0,tbl_9; +CREATE TABLE `tbl_9` ( +`col_54` mediumint NOT NULL DEFAULT '2412996', +`col_55` int NOT NULL, +`col_56` bigint unsigned NOT NULL, +`col_57` varchar(108) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci NOT NULL, +PRIMARY KEY (`col_57`(3),`col_55`,`col_56`,`col_54`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin; +CREATE TABLE `tbl_0` ( +`col_76` bigint(20) unsigned DEFAULT NULL, +`col_1` time NOT NULL DEFAULT '13:11:28', +`col_2` datetime DEFAULT '1990-07-29 00:00:00', +`col_3` date NOT NULL DEFAULT '1976-09-16', +`col_4` date DEFAULT NULL, +`col_143` varbinary(208) DEFAULT 'lXRTXUkTeWaJ', +KEY `idx_0` (`col_2`,`col_1`,`col_76`,`col_4`,`col_3`), +PRIMARY KEY (`col_1`,`col_3`) /*T![clustered_index] NONCLUSTERED */, +KEY `idx_2` (`col_1`,`col_4`,`col_76`,`col_3`), +KEY `idx_3` (`col_4`,`col_76`,`col_3`,`col_2`,`col_1`), +UNIQUE KEY `idx_4` (`col_76`,`col_3`,`col_1`,`col_4`), +KEY `idx_5` (`col_3`,`col_4`,`col_76`,`col_2`), +KEY `idx_6` (`col_2`), +KEY `idx_7` (`col_76`,`col_3`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin; +insert into tbl_9 values (-5765442,-597990898,384599625723370089,"ZdfkUJiHcOfi"); +(select col_76,col_1,col_143,col_2 from tbl_0) union (select col_54,col_57,col_55,col_56 from tbl_9); +col_76 col_1 col_143 col_2 +-5765442 ZdfkUJiHcOfi -597990898 384599625723370089 +drop table if exists t1; +create table t1 (c_int int not null); +insert into t1 values (1), (2), (3),(1),(2),(3); +drop table if exists t2; +create table t2 (c_int int not null); +insert into t2 values (1), (2), (3),(1),(2),(3); +select (select count(distinct c_int) from t2 where c_int >= t1.c_int) c1, (select count(distinct c_int) from t2 where c_int >= t1.c_int) c2 from t1 group by c_int; +c1 c2 +3 3 +2 2 +1 1 +drop table if exists t; +CREATE TABLE `t` ( +`COL1` tinyint(45) NOT NULL, +`COL2` tinyint(45) NOT NULL, +PRIMARY KEY (`COL1`,`COL2`) /*T![clustered_index] NONCLUSTERED */ +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin; +insert into t values(122,100),(124,-22),(124,34),(127,103); +SELECT col2 FROM t AS T1 WHERE ( SELECT count(DISTINCT COL1, COL2) FROM t AS T2 WHERE T2.COL1 > T1.COL1 ) > 2 ; +col2 +100 +select extract(hour_second from "-838:59:59.00"); +extract(hour_second from "-838:59:59.00") +-8385959 +drop table if exists t; +create table t(c1 varchar(100)); +insert into t values('-838:59:59.00'), ('700:59:59.00'); +select extract(hour_second from c1) from t order by c1; +extract(hour_second from c1) +-8385959 +7005959 +select a + 1 as f from (select cast(0xfffffffffffffff0 as unsigned) as a union select cast(1 as unsigned)) t having f != 2; +f +18446744073709551601 +drop table if exists PK_TCOLLATION3966STROBJSTROBJ; +CREATE TABLE `PK_TCOLLATION3966STROBJSTROBJ` ( +`COL1` enum('ll','aa','bb','cc','dd','ee') COLLATE utf8_general_ci NOT NULL, +`COL2` varchar(20) COLLATE utf8_general_ci DEFAULT NULL, +PRIMARY KEY (`COL1`) /*T![clustered_index] CLUSTERED */ +) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_general_ci; +insert into PK_TCOLLATION3966STROBJSTROBJ values("ee", "tttt"); +SELECT col1, COL2 FROM PK_TCOLLATION3966STROBJSTROBJ WHERE COL1 IN ('notexist','6') and col2 not in ("abcd"); +col1 COL2 +drop table if exists perf_offline_day; +CREATE TABLE perf_offline_day ( +uuid varchar(50), +ts timestamp NOT NULL, +user_id varchar(50) COLLATE utf8mb4_general_ci DEFAULT NULL, +platform varchar(50) COLLATE utf8mb4_general_ci DEFAULT NULL, +host_id bigint(20) DEFAULT NULL, +PRIMARY KEY (uuid,ts) /*T![clustered_index] NONCLUSTERED */ +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci +PARTITION BY RANGE ( UNIX_TIMESTAMP(ts) ) ( +PARTITION p20210906 VALUES LESS THAN (1630944000), +PARTITION p20210907 VALUES LESS THAN (1631030400), +PARTITION p20210908 VALUES LESS THAN (1631116800), +PARTITION p20210909 VALUES LESS THAN (1631203200) +); +set @@tidb_partition_prune_mode = 'static'; +INSERT INTO `perf_offline_day` VALUES ('dd082c8a-3bab-4431-943a-348fe0592abd','2021-09-08 13:00:07','Xg9C8zq81jGNbugM', 'pc', 12345); +SELECT cast(floor(hour(ts) / 4) as char) as win_start FROM perf_offline_day partition (p20210907, p20210908) GROUP BY win_start; +win_start +3 +drop table if exists t; +create table t(a time(4)); +insert into t values("-838:59:59.000000"); +insert into t values("838:59:59.000000"); +set tidb_enable_vectorized_expression = on; +select hour(a) from t; +hour(a) +838 +838 +set tidb_enable_vectorized_expression = off; +select hour(a) from t; +hour(a) +838 +838 +set tidb_enable_vectorized_expression = default; +drop table if exists t; +create table t(a enum("a", "b"), b enum("a", "b"), c bool); +insert into t values("a", "a", 1); +select * from t t1 right join t t2 on t1.a=t2.b and t1.a= t2.c; +a b c a b c +a a 1 a a 1 +drop table if exists t; +create table t(a enum("a", "b"), b enum("a", "b"), c bool, d int, index idx(d)); +insert into t values("a", "a", 1, 1); +select /*+ inl_hash_join(t1) */ * from t t1 right join t t2 on t1.a=t2.b and t1.a= t2.c and t1.d=t2.d; +a b c d a b c d +a a 1 1 a a 1 1 +drop table if exists t1; +create table t1(c1 datetime); +insert into t1 values('2021-12-12 10:10:10.000'); +set tidb_enable_vectorized_expression = on; +select greatest(c1, '99999999999999') from t1; +greatest(c1, '99999999999999') +99999999999999 +select least(c1, '99999999999999') from t1; +least(c1, '99999999999999') +2021-12-12 10:10:10 +set tidb_enable_vectorized_expression = off; +select greatest(c1, '99999999999999') from t1; +greatest(c1, '99999999999999') +99999999999999 +select least(c1, '99999999999999') from t1; +least(c1, '99999999999999') +2021-12-12 10:10:10 +set tidb_enable_vectorized_expression = default; +drop table if exists t; +create table t(a time(4)); +insert into t values("-700:10:10.123456111"); +insert into t values("700:10:10.123456111"); +set tidb_enable_vectorized_expression = on; +select microsecond(a) from t; +microsecond(a) +123500 +123500 +set tidb_enable_vectorized_expression = off; +select microsecond(a) from t; +microsecond(a) +123500 +123500 +set tidb_enable_vectorized_expression = default; +set tidb_enable_vectorized_expression = on; +select char(123, NULL, 123); +char(123, NULL, 123) +{{ +select char(NULL, 123, 123); +char(NULL, 123, 123) +{{ +set tidb_enable_vectorized_expression = off; +select char(123, NULL, 123); +char(123, NULL, 123) +{{ +select char(NULL, 123, 123); +char(NULL, 123, 123) +{{ +set tidb_enable_vectorized_expression = default; +drop table if exists t1; +create table t1(c1 bigint unsigned, c2 bigint unsigned); +insert into t1 values(9223372036854775808, 9223372036854775809); +select greatest(c1, c2) from t1; +greatest(c1, c2) +9223372036854775809 +drop table if exists t; +create table t(a int); +insert into t values(1),(1),(2),(2); +set tidb_window_concurrency = 1; +select (FIRST_VALUE(1) over (partition by v.a)) as c3 from (select a from t where t.a = (select a from t t2 where t.a = t2.a)) as v; +Error 1242 (21000): Subquery returns more than 1 row +drop table if exists t1,t2; +CREATE TABLE `t1` ( +`c1` enum('Alice','Bob','Charlie','David') NOT NULL, +`c2` blob NOT NULL, +PRIMARY KEY (`c2`(5)), +UNIQUE KEY `idx_89` (`c1`) +); +CREATE TABLE `t2` ( +`c1` enum('Alice','Bob','Charlie','David') NOT NULL DEFAULT 'Alice', +`c2` set('Alice','Bob','Charlie','David') NOT NULL DEFAULT 'David', +`c3` enum('Alice','Bob','Charlie','David') NOT NULL, +PRIMARY KEY (`c3`,`c2`) +); +insert into t1 values('Charlie',''); +insert into t2 values('Charlie','Charlie','Alice'); +select * from t2 where c3 in (select c2 from t1); +c1 c2 c3 +select * from t2 where c2 in (select c2 from t1); +c1 c2 c3 +drop table if exists t1; +CREATE TABLE t1 (a text)character set utf8 ; +INSERT INTO t1 VALUES (REPEAT(0125,200000000)); +Error 1301 (HY000): Result of repeat() was larger than max_allowed_packet (67108864) - truncated +select * from t1; +a +insert into t1 (a) values ('a'),('b'); +insert into t1 select REPEAT(a,200000000) from t1; +Error 1301 (HY000): Result of repeat() was larger than max_allowed_packet (67108864) - truncated +select a from t1 order by a; +a +a +b +insert into t1 values (cast("a" as binary(4294967295))); +Error 1301 (HY000): Result of cast_as_binary() was larger than max_allowed_packet (67108864) - truncated +select a from t1 order by a; +a +a +b +INSERT IGNORE INTO t1 VALUES (REPEAT(0125,200000000)); +Level Code Message +Warning 1301 Result of repeat() was larger than max_allowed_packet (67108864) - truncated +select a from t1 order by a; +a +NULL +a +b +drop table if exists t; +create table t(a varchar(32)) DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_unicode_ci; +insert into t values('ʞ'), ('İ'); +set @@tidb_enable_vectorized_expression = false; +select binary upper(a), lower(a) from t order by upper(a); +binary upper(a) lower(a) +İ i +Ʞ ʞ +select distinct upper(a), lower(a) from t order by upper(a); +upper(a) lower(a) +İ i +Ʞ ʞ +set @@tidb_enable_vectorized_expression = true; +select binary upper(a), lower(a) from t order by upper(a); +binary upper(a) lower(a) +İ i +Ʞ ʞ +select distinct upper(a), lower(a) from t order by upper(a); +upper(a) lower(a) +İ i +Ʞ ʞ +set @@tidb_enable_vectorized_expression = default; +drop table if exists t; +create table t(a varchar(32)) DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_unicode_ci; +insert into t values(''), (''); +set @@tidb_enable_vectorized_expression = true; +select compress(a) from t; +compress(a) + + +set @@tidb_enable_vectorized_expression = false; +select compress(a) from t; +compress(a) + + +set @@tidb_enable_vectorized_expression = default; +drop table if exists t; +create table t(a varchar(32)); +insert into t values(date_add(cast('00:00:00' as time), interval 1.1 second)); +select * from t; +a +00:00:01.1 +select cast(date_add(cast('00:00:00' as time), interval 1.1 second) as char); +cast(date_add(cast('00:00:00' as time), interval 1.1 second) as char) +00:00:01.1 +select cast(date_add(cast('00:00:00' as time), interval 1.1 microsecond) as char); +cast(date_add(cast('00:00:00' as time), interval 1.1 microsecond) as char) +00:00:00.000001 +select cast(date_add(cast('00:00:00' as time), interval 1000000 microsecond) as char); +cast(date_add(cast('00:00:00' as time), interval 1000000 microsecond) as char) +00:00:01.000000 +select cast(date_add(cast('00:00:00' as time), interval 1.1111119 second) as char); +cast(date_add(cast('00:00:00' as time), interval 1.1111119 second) as char) +00:00:01.111111 +select cast(date_add(cast('00:00:00' as time), interval 1.0 second) as char); +cast(date_add(cast('00:00:00' as time), interval 1.0 second) as char) +00:00:01.0 +select cast(date_add(cast('00:00:00' as time), interval 1.1 second_microsecond) as char); +cast(date_add(cast('00:00:00' as time), interval 1.1 second_microsecond) as char) +00:00:01.100000 +select cast(date_add(cast('00:00:00' as time), interval 1111111 second_microsecond) as char); +cast(date_add(cast('00:00:00' as time), interval 1111111 second_microsecond) as char) +00:00:01.111111 +select cast(date_add(cast('00:00:00' as time), interval 1.1 minute_microsecond) as char); +cast(date_add(cast('00:00:00' as time), interval 1.1 minute_microsecond) as char) +00:00:01.100000 +select cast(date_add(cast('00:00:00' as time), interval 1111111 minute_microsecond) as char); +cast(date_add(cast('00:00:00' as time), interval 1111111 minute_microsecond) as char) +00:00:01.111111 +select cast(date_add(cast('00:00:00' as time), interval 1.1 minute_second) as char); +cast(date_add(cast('00:00:00' as time), interval 1.1 minute_second) as char) +00:01:01 +select cast(date_add(cast('00:00:00' as time), interval 1111111 minute_second) as char); +cast(date_add(cast('00:00:00' as time), interval 1111111 minute_second) as char) +308:38:31 +select cast(date_add(cast('00:00:00' as time), interval 1.1 hour_microsecond) as char); +cast(date_add(cast('00:00:00' as time), interval 1.1 hour_microsecond) as char) +00:00:01.100000 +select cast(date_add(cast('00:00:00' as time), interval 1111111 hour_microsecond) as char); +cast(date_add(cast('00:00:00' as time), interval 1111111 hour_microsecond) as char) +00:00:01.111111 +select cast(date_add(cast('00:00:00' as time), interval 1.1 hour_second) as char); +cast(date_add(cast('00:00:00' as time), interval 1.1 hour_second) as char) +00:01:01 +select cast(date_add(cast('00:00:00' as time), interval 1111111 hour_second) as char); +cast(date_add(cast('00:00:00' as time), interval 1111111 hour_second) as char) +308:38:31 +select cast(date_add(cast('00:00:00' as time), interval 1.1 hour_minute) as char); +cast(date_add(cast('00:00:00' as time), interval 1.1 hour_minute) as char) +01:01:00 +select cast(date_add(cast('00:00:00' as time), interval 1.1 day_microsecond) as char); +cast(date_add(cast('00:00:00' as time), interval 1.1 day_microsecond) as char) +00:00:01.100000 +select cast(date_add(cast('00:00:00' as time), interval 1111111 day_microsecond) as char); +cast(date_add(cast('00:00:00' as time), interval 1111111 day_microsecond) as char) +00:00:01.111111 +drop table if exists t; +create table t(i int, c varchar(32)); +insert into t values(1, date_add(cast('2001-01-01 00:00:00' as datetime), interval 1 second)); +insert into t values(2, date_add(cast('2001-01-01 00:00:00' as datetime(6)), interval 1 second)); +insert into t values(3, date_add(cast('2001-01-01 00:00:00' as datetime), interval 1.1 second)); +insert into t values(4, date_add(cast('2001-01-01 00:00:00' as datetime(6)), interval 1.1 second)); +insert into t values(5, date_add(cast('00:00:00' as time), interval 1.1 second)); +select c from t order by i; +c +2001-01-01 00:00:01 +2001-01-01 00:00:01.000000 +2001-01-01 00:00:01.1 +2001-01-01 00:00:01.100000 +00:00:01.1 +set time_zone = '+00:00'; +drop table if exists t; +create table t(ts timestamp(6) not null default current_timestamp(6) on update current_timestamp(6)); +insert into t values('1970-01-01 01:00:01.000000'); +insert into t values('1970-01-01 01:00:01.000001'); +insert into t values('1971-01-01 01:00:00.000000'); +insert into t values('1971-01-01 01:00:00.000001'); +insert into t values('2001-01-01 00:00:00.000000'); +insert into t values('2001-01-01 00:00:00.000001'); +insert into t values('2001-01-01 01:00:00.000000'); +insert into t values('2001-01-01 01:00:00.000001'); +select date_add(ts, interval 1 minute) from t order by ts; +date_add(ts, interval 1 minute) +1970-01-01 01:01:01.000000 +1970-01-01 01:01:01.000001 +1971-01-01 01:01:00.000000 +1971-01-01 01:01:00.000001 +2001-01-01 00:01:00.000000 +2001-01-01 00:01:00.000001 +2001-01-01 01:01:00.000000 +2001-01-01 01:01:00.000001 +select date_sub(ts, interval 1 minute) from t order by ts; +date_sub(ts, interval 1 minute) +1970-01-01 00:59:01.000000 +1970-01-01 00:59:01.000001 +1971-01-01 00:59:00.000000 +1971-01-01 00:59:00.000001 +2000-12-31 23:59:00.000000 +2000-12-31 23:59:00.000001 +2001-01-01 00:59:00.000000 +2001-01-01 00:59:00.000001 +set time_zone = default; +set time_zone = '+00:00'; +drop table if exists t; +create table t (tm_fsp0 time(0), tm_fsp1 time(1), tm_fsp3 time(3),tm_fsp6 time(6), d date, dt_fsp0 datetime(0), dt_fsp1 datetime(1), dt_fsp3 datetime(3), dt_fsp6 datetime(6)); +insert into t values(null, '12:12:01.1', '12:12:02.123', '12:12:03.123456', '20221212', null, '2020/12/11 12:12:11.1', '2020/12/12 12:12:12.123', '2020/12/13 12:12:13.123456'); +insert into t values('12:12:00', null, '12:12:02.123', '12:12:03.123456', '20221212', '2020/12/10 12:12:10', null, '2020/12/12 12:12:12.123', '2020/12/13 12:12:13.123456'); +insert into t values('12:12:00', '12:12:01.1', null, '12:12:03.123456', '20221212', '2020/12/10 12:12:10', '2020/12/11 12:12:11.1', null, '2020/12/13 12:12:13.123456'); +insert into t values('12:12:00', '12:12:01.1', '12:12:02.123', null, '20221212', '2020/12/10 12:12:10', '2020/12/11 12:12:11.1', '2020/12/12 12:12:12.123', null); +select coalesce(null, tm_fsp0, tm_fsp1, tm_fsp3, tm_fsp6) from t; +coalesce(null, tm_fsp0, tm_fsp1, tm_fsp3, tm_fsp6) +12:12:01.100000 +12:12:00.000000 +12:12:00.000000 +12:12:00.000000 +select coalesce(tm_fsp1, tm_fsp0, tm_fsp3) from t; +coalesce(tm_fsp1, tm_fsp0, tm_fsp3) +12:12:01.100 +12:12:00.000 +12:12:01.100 +12:12:01.100 +select coalesce(tm_fsp3, tm_fsp0) from t; +coalesce(tm_fsp3, tm_fsp0) +12:12:02.123 +12:12:02.123 +12:12:00.000 +12:12:02.123 +select coalesce(tm_fsp6) from t; +coalesce(tm_fsp6) +12:12:03.123456 +12:12:03.123456 +12:12:03.123456 +NULL +select coalesce(null, dt_fsp0, dt_fsp1, dt_fsp3, dt_fsp6) from t; +coalesce(null, dt_fsp0, dt_fsp1, dt_fsp3, dt_fsp6) +2020-12-11 12:12:11.100000 +2020-12-10 12:12:10.000000 +2020-12-10 12:12:10.000000 +2020-12-10 12:12:10.000000 +select coalesce(dt_fsp0, dt_fsp1, dt_fsp3) from t; +coalesce(dt_fsp0, dt_fsp1, dt_fsp3) +2020-12-11 12:12:11.100 +2020-12-10 12:12:10.000 +2020-12-10 12:12:10.000 +2020-12-10 12:12:10.000 +select coalesce(dt_fsp3, dt_fsp0) from t; +coalesce(dt_fsp3, dt_fsp0) +2020-12-12 12:12:12.123 +2020-12-12 12:12:12.123 +2020-12-10 12:12:10.000 +2020-12-12 12:12:12.123 +select coalesce(dt_fsp6) from t; +coalesce(dt_fsp6) +2020-12-13 12:12:13.123456 +2020-12-13 12:12:13.123456 +2020-12-13 12:12:13.123456 +NULL +select coalesce(null, d) from t; +coalesce(null, d) +2022-12-12 +2022-12-12 +2022-12-12 +2022-12-12 +set time_zone = default; +drop table if exists t; +create table t (c int primary key, c2 enum('a', 'b')); +set session sql_mode = ''; +insert into t values(4, 'a'); +insert into t values(4, 0) on duplicate key update c=values(c), c2=values(c2); +Level Code Message +Warning 1265 Data truncated for column 'c2' at row 1 +Warning 1265 Data truncated for column 'c2' at row 1 +insert into t values(4, 'a') on duplicate key update c=values(c), c2=values(c2); +set @@sql_mode=default; +drop table if exists t0, t1; +CREATE TABLE t0(c0 BOOL, c1 INT); +CREATE TABLE t1 LIKE t0; +CREATE definer='root'@'localhost' VIEW v0(c0) AS SELECT IS_IPV4(t0.c1) FROM t0, t1; +INSERT INTO t0(c0, c1) VALUES (true, 0); +INSERT INTO t1(c0, c1) VALUES (true, 2); +SELECT v0.c0 FROM v0 WHERE (v0.c0)NOT LIKE(BINARY v0.c0); +c0 +SELECT v0.c0 FROM v0 WHERE (v0.c0)NOT LIKE(BINARY v0.c0) or v0.c0 > 0; +c0 +select greatest(time '21:00', year(date'20220101'), 23); +greatest(time '21:00', year(date'20220101'), 23) +23 +select greatest(time '21:00', date'891001', 120000); +greatest(time '21:00', date'891001', 120000) +21:00:00 +select greatest(time '20:00', date'101001', 120101); +greatest(time '20:00', date'101001', 120101) +20:00:00 +select greatest(date'101001', '19990329', 120101); +greatest(date'101001', '19990329', 120101) +2012-01-01 +select greatest(time '20:00', date'691231'); +greatest(time '20:00', date'691231') +2069-12-31 00:00:00 +select greatest(date '120301', date'691231'); +greatest(date '120301', date'691231') +2069-12-31 +select greatest(time '203001', time '2230'); +greatest(time '203001', time '2230') +20:30:01 +select greatest(timestamp '2021-01-31 00:00:01', timestamp '2021-12-31 12:00:00'); +greatest(timestamp '2021-01-31 00:00:01', timestamp '2021-12-31 12:00:00') +2021-12-31 12:00:00 +select greatest(time '00:00:01', timestamp '2069-12-31 12:00:00'); +greatest(time '00:00:01', timestamp '2069-12-31 12:00:00') +2069-12-31 12:00:00 +select greatest(date '21000101', timestamp '2069-12-31 12:00:00'); +greatest(date '21000101', timestamp '2069-12-31 12:00:00') +2100-01-01 00:00:00 +select greatest(cast('1' as JSON), cast('2' as JSON)); +greatest(cast('1' as JSON), cast('2' as JSON)) +2 +select greatest(time '20:00:00', 120000); +greatest(time '20:00:00', 120000) +20:00:00 +select greatest(date '2005-05-05', 20010101, 20040404, 20030303); +greatest(date '2005-05-05', 20010101, 20040404, 20030303) +2005-05-05 +select greatest(date '1995-05-05', 19910101, 20050505, 19930303); +greatest(date '1995-05-05', 19910101, 20050505, 19930303) +2005-05-05 +drop table if exists t1,t2; +CREATE TABLE `t1` (a datetime, b date, c time); +insert into t1 values(timestamp'2021-01-31 00:00:01', '2069-12-31', '20:00:01'); +set tidb_enable_vectorized_expression = on; +select greatest(c, year(date'20220101'), 23) from t1; +greatest(c, year(date'20220101'), 23) +23 +select greatest(c, date'891001', 120000) from t1; +greatest(c, date'891001', 120000) +20:00:01 +select greatest(c, date'101001', 120101) from t1; +greatest(c, date'101001', 120101) +20:00:01 +select greatest(b, '19990329', 120101) from t1; +greatest(b, '19990329', 120101) +2069-12-31 +select greatest(time '20:00', b) from t1; +greatest(time '20:00', b) +2069-12-31 00:00:00 +select greatest(date '120301', b) from t1; +greatest(date '120301', b) +2069-12-31 +select greatest(c, time '2230') from t1; +greatest(c, time '2230') +20:00:01 +select greatest(a, timestamp '2021-12-31 12:00:00') from t1; +greatest(a, timestamp '2021-12-31 12:00:00') +2021-12-31 12:00:00 +select greatest(c, timestamp '2069-12-31 12:00:00') from t1; +greatest(c, timestamp '2069-12-31 12:00:00') +2069-12-31 12:00:00 +select greatest(date '21000101', a) from t1; +greatest(date '21000101', a) +2100-01-01 00:00:00 +select greatest(cast(a as JSON), cast('3' as JSON)) from t1; +greatest(cast(a as JSON), cast('3' as JSON)) +3 +set tidb_enable_vectorized_expression = default; +set tidb_enable_prepared_plan_cache=ON; +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 (id INT NOT NULL PRIMARY KEY auto_increment, a timestamp NOT NULL); +INSERT INTO t1 VALUES (null, '2020-05-30 20:30:00'); +PREPARE mystmt FROM 'SELECT * FROM t1 WHERE UNIX_TIMESTAMP(a) >= ?'; +SET @a=1590868800; +EXECUTE mystmt USING @a; +id a +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +SET @a=1590868801; +EXECUTE mystmt USING @a; +id a +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +prepare stmt from 'select unix_timestamp(?)'; +set @a = '2020-05-30 20:30:00'; +execute stmt using @a; +unix_timestamp(?) +1590841800 +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +set @a = '2020-06-12 13:47:58'; +execute stmt using @a; +unix_timestamp(?) +1591940878 +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +drop table if exists t; +set sql_mode=default; +create table t(id int, value set ('a','b','c') charset utf8mb4 collate utf8mb4_bin default 'a,b '); +drop table if exists test; +create table test(id int, value set ('a','b','c') charset utf8mb4 collate utf8mb4_general_ci default 'a,B ,C'); +drop table if exists t; +create table t(a char(4) collate utf8_general_ci primary key /*T![clustered_index] clustered */); +insert into t values('`?'); +explain format='brief' select * from t where a like '`%'; +id estRows task access object operator info +TableReader 250.00 root data:Selection +└─Selection 250.00 cop[tikv] like(expression__issues.t.a, "`%", 92) + └─TableRangeScan 250.00 cop[tikv] table:t range:["\x00`","\x00a"), keep order:false, stats:pseudo +select * from t where a like '`%'; +a +`? +drop table if exists t; +create table t(a char(4) collate binary primary key /*T![clustered_index] clustered */); +insert into t values('`?'); +explain format='brief' select * from t where a like '`%'; +id estRows task access object operator info +TableReader 250.00 root data:TableRangeScan +└─TableRangeScan 250.00 cop[tikv] table:t range:["`","a"), keep order:false, stats:pseudo +select * from t where a like '`%'; +a +`? +drop table if exists t; +CREATE TABLE `t` ( `a` enum('a','b') DEFAULT NULL ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci; +insert into t values('a'); +select * from t where a = 'A'; +a +a +drop table if exists tx; +CREATE TABLE `tx` ( `a` int(11) NOT NULL,`b` varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci DEFAULT NULL); +insert into tx values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' '); +select count(distinct(b)) from tx; +count(distinct(b)) +4 +drop table if exists tx; +CREATE TABLE `tx` ( `a` int(11) NOT NULL,`b` varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci DEFAULT NULL, `c` varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin DEFAULT NULL); +insert into tx values (1, 'a', 'a'), (2, 'A ', 'a '), (3, 'A', 'A'), (4, 'a ', 'A '); +select count(distinct b) from tx; +count(distinct b) +1 +select count(distinct c) from tx; +count(distinct c) +2 +select count(distinct b, c) from tx where a < 3; +count(distinct b, c) +1 +select approx_count_distinct(b) from tx; +approx_count_distinct(b) +1 +select approx_count_distinct(c) from tx; +approx_count_distinct(c) +2 +select approx_count_distinct(b, c) from tx where a < 3; +approx_count_distinct(b, c) +1 +drop table if exists t; +create table t(a enum('a', 'a ')) charset utf8 collate utf8_bin; +Error 1291 (HY000): Column 'a' has duplicated value 'a' in ENUM +create table t(a enum('a', 'Á')) charset utf8 collate utf8_general_ci; +Error 1291 (HY000): Column 'a' has duplicated value 'Á' in ENUM +create table t(a enum('a', 'a ')) charset utf8mb4 collate utf8mb4_bin; +Error 1291 (HY000): Column 'a' has duplicated value 'a' in ENUM +create table t(a enum('a', 'A')) charset utf8 collate utf8_bin; +drop table if exists t3; +create table t3(a enum('a', 'A')) charset utf8mb4 collate utf8mb4_bin; +drop table if exists t; +create table t(a varchar(10) collate utf8mb4_bin, b varchar(10) collate utf8mb4_general_ci); +insert into t (a, b) values ('a', 'A'); +select * from t t1, t t2 where t1.a = t2.b collate utf8mb4_general_ci; +a b a b +a A a A +select * from t t1 left join t t2 on t1.a = t2.b collate utf8mb4_general_ci; +a b a b +a A a A +set names utf8mb4 collate utf8mb4_general_ci; +select collation(concat(1 collate `binary`)); +collation(concat(1 collate `binary`)) +binary +select coercibility(concat(1 collate `binary`)); +coercibility(concat(1 collate `binary`)) +0 +select collation(concat(NULL,NULL)); +collation(concat(NULL,NULL)) +binary +select coercibility(concat(NULL,NULL)); +coercibility(concat(NULL,NULL)) +6 +select collation(concat(1,1)); +collation(concat(1,1)) +utf8mb4_general_ci +select coercibility(concat(1,1)); +coercibility(concat(1,1)) +4 +select collation(1); +collation(1) +binary +select coercibility(1); +coercibility(1) +5 +select coercibility(1=1); +coercibility(1=1) +5 +drop table if exists t; +create table t(a datetime); +insert into t values ('2020-02-02'); +select collation(concat(unix_timestamp(a))) from t; +collation(concat(unix_timestamp(a))) +utf8mb4_general_ci +select coercibility(concat(unix_timestamp(a))) from t; +coercibility(concat(unix_timestamp(a))) +4 +set names default; +drop table if exists t; +create table t(a char, b char) collate utf8mb4_general_ci; +insert into t values('a', 'b'); +insert into t values('a', 'B'); +select * from t where if(a='x', a, b) = 'b'; +a b +a b +a B +select collation(if(a='x', a, b)) from t; +collation(if(a='x', a, b)) +utf8mb4_general_ci +utf8mb4_general_ci +select coercibility(if(a='x', a, b)) from t; +coercibility(if(a='x', a, b)) +2 +2 +select collation(lag(b, 1, 'B') over w) from t window w as (order by b); +collation(lag(b, 1, 'B') over w) +utf8mb4_general_ci +utf8mb4_general_ci +select coercibility(lag(b, 1, 'B') over w) from t window w as (order by b); +coercibility(lag(b, 1, 'B') over w) +2 +2 +SELECT 'lvuleck' BETWEEN '2008-09-16 22:23:50' AND 0; +'lvuleck' BETWEEN '2008-09-16 22:23:50' AND 0 +0 +Level Code Message +Warning 1292 Truncated incorrect DOUBLE value: '2008-09-16 22:23:50' +Warning 1292 Truncated incorrect DOUBLE value: 'lvuleck' +SELECT 'aa' BETWEEN 'bb' AND 0; +'aa' BETWEEN 'bb' AND 0 +1 +Level Code Message +Warning 1292 Truncated incorrect DOUBLE value: 'aa' +Warning 1292 Truncated incorrect DOUBLE value: 'bb' +select 1 between 0 and b'110'; +1 between 0 and b'110' +1 +select 'b' between 'a' and b'110'; +'b' between 'a' and b'110' +0 +drop table if exists t; +create table t(a set('a', 'b', 'c')); +alter table t change a a set('a', 'b', 'c', 'c'); +Error 1291 (HY000): Column 'a' has duplicated value 'c' in SET +drop table if exists t; +create table t(a enum('a', 'b', 'c')); +alter table t change a a enum('a', 'b', 'c', 'c'); +Error 1291 (HY000): Column 'a' has duplicated value 'c' in ENUM +drop table if exists t; +create table t(a set('a', 'b', 'c')); +alter table t change a a set('a', 'b', 'c', 'd'); +insert into t values('d'); +alter table t change a a set('a', 'b', 'c', 'e', 'f'); +Error 1265 (01000): Data truncated for column 'a', value is 'd' +set @@character_set_client=utf8mb4; +set @@collation_connection=utf8_bin; +CREATE VIEW tview_1 AS SELECT 'a' AS `id`; +set @@character_set_client=default; +set @@collation_connection=default; +drop table if exists t; +create table t(a enum('a ', 'b\t', ' c '), b set('a ', 'b\t', ' c ')); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` enum('a','b ',' c') DEFAULT NULL, + `b` set('a','b ',' c') DEFAULT NULL +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +alter table t change a aa enum('a ', 'b\t', ' c '); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `aa` enum('a','b ',' c') DEFAULT NULL, + `b` set('a','b ',' c') DEFAULT NULL +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +select '䇇Հ' collate utf8mb4_bin like '___Հ'; +'䇇Հ' collate utf8mb4_bin like '___Հ' +0 +drop table if exists t; +create table t(raw JSON); +insert into t(raw) values('["a","ab"]'), ('["a"]'), (null); +SELECT JSON_SEARCH(raw,'one','c') FROM t; +JSON_SEARCH(raw,'one','c') +NULL +NULL +NULL +drop table if exists t; +CREATE TABLE `t` ( `a` char(10) COLLATE utf8mb4_unicode_ci NOT NULL, `b` char(20) COLLATE utf8mb4_general_ci NOT NULL, `c` int(11) NOT NULL, PRIMARY KEY (`a`,`b`,`c`), KEY `idx` (`a`)); +insert into t values ('#', 'C', 10), ('$', 'c', 20), ('$', 'c', 30), ('a', 'a', 10), ('A', 'A', 30); +analyze table t; +select * from t where a='#'; +a b c +# C 10 +CREATE TABLE `tbl_5` ( `col_25` time NOT NULL DEFAULT '05:35:58', `col_26` blob NOT NULL, `col_27` double NOT NULL, `col_28` char(83) CHARACTER SET utf8 COLLATE utf8_bin NOT NULL, `col_29` timestamp NOT NULL, `col_30` varchar(36) COLLATE utf8mb4_general_ci NOT NULL DEFAULT 'ywzIn', `col_31` binary(85) DEFAULT 'OIstcXsGmAyc', `col_32` datetime NOT NULL DEFAULT '2024-08-02 00:00:00', PRIMARY KEY (`col_26`(3),`col_27`) /*T![clustered_index] CLUSTERED */, UNIQUE KEY `idx_10` (`col_26`(5))); +insert ignore into tbl_5 set col_28 = 'ZmZIdSnq' , col_25 = '18:50:52.00' on duplicate key update col_26 = 'y'; +drop table if exists t1; +create table t1(a varchar(36) NOT NULL) ENGINE = InnoDB DEFAULT CHARSET = utf8 COLLATE = utf8_general_ci; +set names utf8; +select t2.b from (select t1.a as b from t1 union all select t1.a as b from t1) t2 where case when (t2.b is not null) then t2.b else '' end > '1234567'; +b +set names default; +select case 1 when 1 then 'a' collate utf8mb4_unicode_ci else 'b' collate utf8mb4_general_ci end; +Error 1267 (HY000): Illegal mix of collations (utf8mb4_unicode_ci,EXPLICIT) and (utf8mb4_general_ci,EXPLICIT) for operation 'case' +select case when 1 then 'a' collate utf8mb4_unicode_ci when 2 then 'b' collate utf8mb4_general_ci end; +Error 1267 (HY000): Illegal mix of collations (utf8mb4_unicode_ci,EXPLICIT) and (utf8mb4_general_ci,EXPLICIT) for operation 'case' +select case 1 when 1 then 'a' collate utf8mb4_unicode_ci when 2 then 'b' collate utf8mb4_general_ci else 'b' collate utf8mb4_bin end; +Error 1270 (HY000): Illegal mix of collations (utf8mb4_unicode_ci,EXPLICIT), (utf8mb4_general_ci,EXPLICIT), (utf8mb4_bin,EXPLICIT) for operation 'case' +drop table if exists t0,t1; +create table t0(col1 varchar(255)); +create table t1(col1 int(11)); +insert into t0 values("\\9jM\\M"); +insert into t1 values(0); +insert into t1 values(null); +insert into t1 values(null); +select t0.col1, t1.col1 from t0 left join t1 on t0.col1 not like t0.col1; +col1 col1 +\9jM\M NULL +\9jM\M NULL +\9jM\M 0 +select 'a' like '\\a'; +'a' like '\\a' +1 +select 'a' like '+a' escape '+'; +'a' like '+a' escape '+' +1 +drop table if exists table_int; +CREATE TABLE table_int ( +id_0 int(16) NOT NULL AUTO_INCREMENT, +col_int_0 int(16) DEFAULT NULL, +PRIMARY KEY (id_0), +KEY fvclc (id_0,col_int_0)); +INSERT INTO table_int VALUES (1,NULL),(2,NULL),(3,65535),(4,1),(5,0),(6,NULL),(7,-1),(8,65535),(9,NULL),(10,65535),(11,-1),(12,0),(13,-1),(14,1),(15,65535),(16,0),(17,1),(18,0),(19,0); +drop table if exists table_varchar; +CREATE TABLE table_varchar ( +id_2 int(16) NOT NULL AUTO_INCREMENT, +col_varchar_2 varchar(511) DEFAULT NULL, +PRIMARY KEY (id_2)); +INSERT INTO table_varchar VALUES (1,''),(2,''),(3,''),(4,''),(5,''),(6,''),(7,''),(8,''),(9,''),(10,''),(11,''),(12,''); +drop table if exists table_int_float_varchar; +CREATE TABLE table_int_float_varchar ( +id_6 int(16) NOT NULL AUTO_INCREMENT, +col_int_6 int(16) NOT NULL, +col_float_6 float DEFAULT NULL, +col_varchar_6 varchar(511) DEFAULT NULL, +PRIMARY KEY (id_6,col_int_6) +) +PARTITION BY RANGE ( col_int_6 ) ( +PARTITION p0 VALUES LESS THAN (1), +PARTITION p2 VALUES LESS THAN (1000), +PARTITION p3 VALUES LESS THAN (10000), +PARTITION p5 VALUES LESS THAN (1000000), +PARTITION p7 VALUES LESS THAN (100000000), +PARTITION p9 VALUES LESS THAN (10000000000), +PARTITION p10 VALUES LESS THAN (100000000000), +PARTITION pn VALUES LESS THAN (MAXVALUE)); +INSERT INTO table_int_float_varchar VALUES (1,-1,0.1,'0000-00-00 00:00:00'),(2,0,0,NULL),(3,-1,1,NULL),(4,0,NULL,NULL),(7,0,0.5,NULL),(8,0,0,NULL),(10,-1,0,'-1'),(5,1,-0.1,NULL),(6,1,0.1,NULL),(9,65535,0,'1'); +drop table if exists table_float; +CREATE TABLE table_float ( +id_1 int(16) NOT NULL AUTO_INCREMENT, +col_float_1 float DEFAULT NULL, +PRIMARY KEY (id_1), +KEY zbjus (id_1,col_float_1)); +INSERT INTO table_float VALUES (1,NULL),(2,-0.1),(3,-1),(4,NULL),(5,-0.1),(6,0),(7,0),(8,-1),(9,NULL),(10,NULL),(11,0.1),(12,-1); +drop view if exists view_4; +CREATE DEFINER='root'@'127.0.0.1' VIEW view_4 (col_1, col_2, col_3, col_4, col_5, col_6, col_7, col_8, col_9, col_10) AS +SELECT /*+ USE_INDEX(table_int fvclc, fvclc)*/ +tmp1.id_6 AS col_1, +tmp1.col_int_6 AS col_2, +tmp1.col_float_6 AS col_3, +tmp1.col_varchar_6 AS col_4, +tmp2.id_2 AS col_5, +tmp2.col_varchar_2 AS col_6, +tmp3.id_0 AS col_7, +tmp3.col_int_0 AS col_8, +tmp4.id_1 AS col_9, +tmp4.col_float_1 AS col_10 +FROM (( +expression__issues.table_int_float_varchar AS tmp1 LEFT JOIN +expression__issues.table_varchar AS tmp2 ON ((NULL<=tmp2.col_varchar_2)) IS NULL +) JOIN +expression__issues.table_int AS tmp3 ON (1.117853833115198e-03!=tmp1.col_int_6)) +JOIN +expression__issues.table_float AS tmp4 ON !((1900370398268920328=0e+00)) WHERE ((''<='{Gm~PcZNb') OR (tmp2.id_2 OR tmp3.col_int_0)) ORDER BY col_1,col_2,col_3,col_4,col_5,col_6,col_7,col_8,col_9,col_10 LIMIT 20580,5; +drop view if exists view_10; +CREATE DEFINER='root'@'127.0.0.1' VIEW view_10 (col_1, col_2) AS +SELECT table_int.id_0 AS col_1, +table_int.col_int_0 AS col_2 +FROM expression__issues.table_int +WHERE +((-1e+00=1) OR (0e+00>=table_int.col_int_0)) +ORDER BY col_1,col_2 +LIMIT 5,9; +SELECT col_1 FROM expression__issues.view_10; +col_1 +16 +18 +19 +SELECT col_1 FROM expression__issues.view_4; +col_1 +8 +8 +8 +8 +8 +SELECT view_10.col_1 FROM view_4 JOIN view_10; +col_1 +16 +16 +16 +16 +16 +18 +18 +18 +18 +18 +19 +19 +19 +19 +19 +drop table if exists t; +create table t(a int, b tinyint as(a+1), c int as(b+1)); +set sql_mode=''; +insert into t(a) values(2000); +create index idx on t(c); +select c from t; +c +128 +admin check table t; +set @@sql_mode=default; +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 ( `pk` int not null primary key auto_increment, `col_smallint_key_signed` smallint , key (`col_smallint_key_signed`)); +INSERT INTO `t1` VALUES (1,0),(2,NULL),(3,NULL),(4,0),(5,0),(6,NULL),(7,NULL),(8,0),(9,0),(10,0); +SELECT * FROM t1 WHERE ( LOG( `col_smallint_key_signed`, -8297584758403770424 ) ) DIV 1; +pk col_smallint_key_signed +drop table if exists t, t1, t2; +CREATE TABLE t ( +id int(11) NOT NULL AUTO_INCREMENT, +a char(10) DEFAULT NULL, +PRIMARY KEY (id) +); +CREATE TABLE t1 ( +id int(11) NOT NULL AUTO_INCREMENT, +a char(10) DEFAULT NULL, +b char(10) DEFAULT NULL, +c char(10) DEFAULT NULL, +PRIMARY KEY (id) +); +CREATE TABLE t2 ( +id int(11) NOT NULL AUTO_INCREMENT, +a char(10) DEFAULT NULL, +b char(10) DEFAULT NULL, +PRIMARY KEY (id), +UNIQUE KEY b (b) +); +insert into t1(a,b,c) values('hs4_0004', "04", "101"), ('a01', "01", "101"),('a011', "02", "101"); +insert into t2(a,b) values("02","03"); +insert into t(a) values('101'),('101'); +select ( SELECT t1.a FROM t1, t2 WHERE t1.b = t2.a AND t2.b = '03' AND t1.c = a.a) invode from t a ; +invode +a011 +a011 +drop table if exists t, t1; +CREATE TABLE `t` (`a` bit(10) DEFAULT NULL,`b` int(11) DEFAULT NULL) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin; +INSERT INTO `t` VALUES (_binary '\0',1),(_binary '\0',2),(_binary '\0',5),(_binary '\0',4),(_binary '\0',2),(_binary '\0 ',4); +CREATE TABLE `t1` (`a` int(11) DEFAULT NULL, `b` int(11) DEFAULT NULL) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin; +INSERT INTO `t1` VALUES (1,1),(1,5),(2,3),(2,4),(3,3); +select * from t where t.b > (select min(t1.b) from t1 where t1.a > t.a); +a b + 2 + 5 + 4 + 2 +drop table if exists t1; +create table t1(a date); +insert into t1 values (20100202); +select a in ('2020-02-02', 20100202) from t1; +a in ('2020-02-02', 20100202) +1 +drop table if exists t; +create table t(b enum('a','b','c','d','e','f','g','h','i','j','k','l','m','n','o','p','q','r','s','t','u','v','w','x','y','z') DEFAULT NULL, c decimal(40,20)); +insert into t values('z', 19.18040000000000000000); +insert into t values('z', 26.18040000000000000000); +insert into t values('z', 25.18040000000000000000); +select * from t where t.b > t.c; +b c +z 19.18040000000000000000 +z 25.18040000000000000000 +select * from t where t.b < t.c; +b c +z 26.18040000000000000000 +drop table if exists t1; +CREATE TABLE t1 (c1 INT); +INSERT INTO t1 VALUES (1), (null); +SELECT IFNULL(c1, 0.0) from t1; +IFNULL(c1, 0.0) +1.0 +0.0 +SELECT if(c1 is not null, c1, 0.0) from t1; +if(c1 is not null, c1, 0.0) +1.0 +0.0 +SELECT case when c1 is not null then c1 else 0.0 end from t1; +case when c1 is not null then c1 else 0.0 end +1.0 +0.0 +drop table if exists t1; +CREATE TABLE t1 (v1 datetime(3), v2 datetime(4)); +INSERT INTO t1 VALUES ('2020-01-01 11:11:11.123', null), (null, '2020-01-01 11:11:11.1234'); +SELECT IFNULL(v1, v2) from t1; +IFNULL(v1, v2) +2020-01-01 11:11:11.1230 +2020-01-01 11:11:11.1234 +SELECT if(v1 is not null, v1, v2) from t1; +if(v1 is not null, v1, v2) +2020-01-01 11:11:11.1230 +2020-01-01 11:11:11.1234 +SELECT case when v1 is not null then v1 else v2 end from t1; +case when v1 is not null then v1 else v2 end +2020-01-01 11:11:11.1230 +2020-01-01 11:11:11.1234 +drop table if exists t1; +CREATE TABLE t1 (v1 timestamp(3), v2 timestamp(4)); +INSERT INTO t1 VALUES ('2020-01-01 11:11:11.123', null), (null, '2020-01-01 11:11:11.1234'); +SELECT IFNULL(v1, v2) from t1; +IFNULL(v1, v2) +2020-01-01 11:11:11.1230 +2020-01-01 11:11:11.1234 +SELECT if(v1 is not null, v1, v2) from t1; +if(v1 is not null, v1, v2) +2020-01-01 11:11:11.1230 +2020-01-01 11:11:11.1234 +SELECT case when v1 is not null then v1 else v2 end from t1; +case when v1 is not null then v1 else v2 end +2020-01-01 11:11:11.1230 +2020-01-01 11:11:11.1234 +drop table if exists t1; +CREATE TABLE t1 (v1 time(3), v2 time(4)); +INSERT INTO t1 VALUES ('11:11:11.123', null), (null, '11:11:11.1234'); +SELECT IFNULL(v1, v2) from t1; +IFNULL(v1, v2) +11:11:11.1230 +11:11:11.1234 +SELECT if(v1 is not null, v1, v2) from t1; +if(v1 is not null, v1, v2) +11:11:11.1230 +11:11:11.1234 +SELECT case when v1 is not null then v1 else v2 end from t1; +case when v1 is not null then v1 else v2 end +11:11:11.1230 +11:11:11.1234 +drop table if exists t1; +CREATE TABLE t1 (var_fld int(11) DEFAULT '0' ); +INSERT INTO t1 VALUES (4500), (6000); +SELECT IF ( var_fld > 5000, ( 00001 / var_fld ) , 5000 ) if_var FROM t1; +if_var +5000.0000 +0.0002 +SELECT CASE WHEN var_fld > 5000 THEN ( 1 / var_fld ) ELSE 5000 END case_var FROM t1; +case_var +5000.0000 +0.0002 +SELECT CASE var_fld > 5000 WHEN TRUE THEN ( 1 / var_fld ) ELSE 5000 END case_var FROM t1; +case_var +5000.0000 +0.0002 +drop table if exists t1; +CREATE TABLE t1 (c1 TINYINT(1) UNSIGNED NOT NULL); +INSERT INTO t1 VALUES (0); +SELECT c1>=CAST('-787360724' AS TIME) FROM t1; +c1>=CAST('-787360724' AS TIME) +1 +select '123' union select cast(45678 as char); +123 +123 +45678 +select '123' union select cast(45678 as char(2)); +123 +123 +45 +drop table if exists t; +create table t(a int); +insert into t values(45678); +select '123' union select cast(a as char) from t; +123 +123 +45678 +select '123' union select cast(a as char(2)) from t; +123 +123 +45 +SET time_zone='Europe/Vilnius'; +SELECT UNIX_TIMESTAMP('2020-03-29 03:45:00'); +UNIX_TIMESTAMP('2020-03-29 03:45:00') +1585443600 +SELECT FROM_UNIXTIME(UNIX_TIMESTAMP('2020-03-29 03:45:00')); +FROM_UNIXTIME(UNIX_TIMESTAMP('2020-03-29 03:45:00')) +2020-03-29 04:00:00 +DROP TABLE IF EXISTS t; +CREATE TABLE t (dt DATETIME NULL); +INSERT INTO t VALUES ('2021-10-31 02:30:00'), ('2021-03-28 02:30:00'), ('2020-10-04 02:15:00'), ('2020-03-29 03:45:00'), (NULL); +SELECT dt, UNIX_TIMESTAMP(dt) FROM t; +dt UNIX_TIMESTAMP(dt) +NULL NULL +2020-03-29 03:45:00 1585443600 +2020-10-04 02:15:00 1601766900 +2021-03-28 02:30:00 1616891400 +2021-10-31 02:30:00 1635636600 +SET time_zone=default; +DROP TABLE IF EXISTS test; +CREATE TABLE test ( c1 varchar(20)); +INSERT INTO test VALUES (101111),(11100),(101111),(101111); +set tidb_enable_vectorized_expression = true; +SELECT DATE_ADD(c1, INTERVAL 1 DAY_HOUR) from test; +DATE_ADD(c1, INTERVAL 1 DAY_HOUR) +NULL +2010-11-11 01:00:00 +2010-11-11 01:00:00 +2010-11-11 01:00:00 +set tidb_enable_vectorized_expression = default; +drop table if exists t; +create table t(a timestamp) partition by range(unix_timestamp(a)) (partition p0 values less than(unix_timestamp('2019-02-16 14:20:00')), partition p1 values less than (maxvalue)); +insert into t values('2019-02-16 14:19:59'), ('2019-02-16 14:20:01'); +select * from t where a between timestamp'2019-02-16 14:19:00' and timestamp'2019-02-16 14:21:00'; +a +2019-02-16 14:19:59 +2019-02-16 14:20:01 +drop table if exists t; +create table t(a timestamp); +insert into t values('2019-02-16 14:19:59'), ('2019-02-16 14:20:01'); +select * from t where a < timestamp'2019-02-16 14:21:00'; +a +2019-02-16 14:19:59 +2019-02-16 14:20:01 +drop table if exists t; +CREATE TABLE t(c varchar(100), index idx(c(100))); +INSERT INTO t VALUES (NULL),('1'),('0'),(''),('aaabbb'),('0abc'),('123e456'),('0.0001deadsfeww'); +select * from t where c; +c +0.0001deadsfeww +1 +123e456 +select /*+ USE_INDEX(t, idx) */ * from t where c; +c +0.0001deadsfeww +1 +123e456 +select /*+ IGNORE_INDEX(t, idx) */* from t where c; +c +0.0001deadsfeww +1 +123e456 +drop table if exists t0; +create table t0 (c1 DATE, c2 TIME, c3 DATETIME, c4 TIMESTAMP); +insert into t0 values ('1000-01-01', '-838:59:59', '1000-01-01 00:00:00', '1970-01-01 08:00:01'); +insert into t0 values ('9999-12-31', '838:59:59', '9999-12-31 23:59:59', '2038-01-19 11:14:07'); +select avg(c1), avg(c2), avg(c3), avg(c4) from t0; +avg(c1) avg(c2) avg(c3) avg(c4) +54995666.0000 0.0000 54995666117979.5000 20040110095704.0000 +select -1.0 % -1.0; +-1.0 % -1.0 +0.0 +drop table if exists t1; +create table t1(`pk` int primary key,`col_float_key_signed` float ,key (`col_float_key_signed`)); +insert into t1 values (0, null), (1, 0), (2, -0), (3, 1), (-1,-1); +select * from t1 where ( `col_float_key_signed` % `col_float_key_signed`) IS FALSE; +pk col_float_key_signed +-1 -1 +3 1 +select `col_float_key_signed` , `col_float_key_signed` % `col_float_key_signed` from t1; +col_float_key_signed `col_float_key_signed` % `col_float_key_signed` +NULL NULL +-1 -0 +0 NULL +0 NULL +1 0 +select `col_float_key_signed` , (`col_float_key_signed` % `col_float_key_signed`) IS FALSE from t1; +col_float_key_signed (`col_float_key_signed` % `col_float_key_signed`) IS FALSE +NULL 0 +-1 1 +0 0 +0 0 +1 1 +drop table if exists author_addresses, authors; +CREATE TABLE author_addresses ( +id bigint(20) NOT NULL AUTO_INCREMENT, +PRIMARY KEY (id) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4; +CREATE TABLE authors ( +id bigint(20) NOT NULL AUTO_INCREMENT, +name varchar(255) NOT NULL, +author_address_id bigint(20) DEFAULT NULL, +author_address_extra_id bigint(20) DEFAULT NULL, +organization_id varchar(255) DEFAULT NULL, +owned_essay_id varchar(255) DEFAULT NULL, +PRIMARY KEY (id), +KEY index_authors_on_author_address_id (author_address_id), +KEY index_authors_on_author_address_extra_id (author_address_extra_id), +CONSTRAINT fk_rails_94423a17a3 FOREIGN KEY (author_address_id) REFERENCES author_addresses (id) ON UPDATE CASCADE ON DELETE RESTRICT +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4; +SELECT fk.referenced_table_name AS 'to_table', +fk.referenced_column_name AS 'primary_key', +fk.column_name AS 'column', +fk.constraint_name AS 'name', +rc.update_rule AS 'on_update', +rc.delete_rule AS 'on_delete' +FROM information_schema.referential_constraints rc +JOIN information_schema.key_column_usage fk +USING (constraint_schema, constraint_name) +WHERE fk.referenced_column_name IS NOT NULL +AND fk.table_schema = database() +AND fk.table_name = 'authors'; +to_table primary_key column name on_update on_delete +author_addresses id author_address_id fk_rails_94423a17a3 CASCADE RESTRICT +drop table if exists t; +CREATE TABLE t (a CHAR(5) CHARACTER SET latin1); +INSERT INTO t VALUES ('oe'); +INSERT INTO t VALUES (0xf6); +SELECT * FROM t WHERE a= 'oe'; +a +oe +SELECT HEX(a) FROM t WHERE a= 0xf6; +HEX(a) +F6 +drop table if exists tb; +create table tb(id int auto_increment primary key, v varchar(32)); +insert into tb(v) (select v from tb); +SELECT * FROM tb; +id v +insert into tb(v) values('hello'); +insert into tb(v) (select v from tb); +SELECT * FROM tb; +id v +1 hello +2 hello +drop table if exists t; +create table t(c varchar(32)); +insert into t values('1e649'),('-1e649'); +SELECT * FROM t where c < 1; +c +-1e649 +SELECT * FROM t where c > 1; +c +1e649 +drop table if exists t; +create table t(a int); +insert t values (1); +select * from t where cast(a as binary); +a +1 +drop table if exists t2; +create table t2(a int, b varchar(20)); +insert into t2 values(1,"1111"),(2,"2222"),(3,"3333"),(4,"4444"),(5,"5555"),(6,"6666"),(7,"7777"),(8,"8888"),(9,"9999"),(10,"0000"); +select (@j := case when substr(t2.b,1,3)=@i then 1 else @j+1 end) from t2, (select @j := 0, @i := "0") tt limit 10; +(@j := case when substr(t2.b,1,3)=@i then 1 else @j+1 end) +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +select b'10000000' DIV 10; +b'10000000' DIV 10 +12 +select cast(b'10000000' as unsigned) / 10; +cast(b'10000000' as unsigned) / 10 +12.8000 +select b'10000000' / 10; +b'10000000' / 10 +12.8000 +drop table if exists t0; +create table t0 (c0 int); +select * from (select (92 / 4) as c4) as subq_0 where exists ( +select 1 as c0 +union all +select 1 as c0 from (t0 as ref_88) where (subq_0.c4) >= (subq_0.c4) +); +c4 +23.0000 +drop table if exists poi_clearing_time_topic; +CREATE TABLE poi_clearing_time_topic (effective_date datetime DEFAULT NULL , clearing_time int(11) DEFAULT NULL); +insert into poi_clearing_time_topic values ('2023:08:25', 1); +SELECT GROUP_CONCAT(effective_date order by stlmnt_hour DESC) FROM ( SELECT (COALESCE(pct.clearing_time, 0)/3600000) AS stlmnt_hour ,COALESCE(pct.effective_date, '1970-01-01 08:00:00') AS effective_date FROM poi_clearing_time_topic pct ORDER BY pct.effective_date DESC ) a; +GROUP_CONCAT(effective_date order by stlmnt_hour DESC) +2023-08-25 00:00:00 +SELECT @@information_schema_stats_expiry; +@@information_schema_stats_expiry +86400 +/*!80000 SET SESSION information_schema_stats_expiry=0 */; +SELECT @@information_schema_stats_expiry; +@@information_schema_stats_expiry +0 +SELECT @@GLOBAL.information_schema_stats_expiry; +@@GLOBAL.information_schema_stats_expiry +86400 +/*!80000 SET GLOBAL information_schema_stats_expiry=0 */; +SELECT @@GLOBAL.information_schema_stats_expiry; +@@GLOBAL.information_schema_stats_expiry +0 +set @@SESSION.information_schema_stats_expiry=default; +set @@GLOBAL.information_schema_stats_expiry=default; +drop table if exists t; +drop table if exists t1; +create table t(col1 decimal); +insert into t values(0.00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000); +select * from t; +col1 +0 +create table t1(col1 decimal(65,30)); +insert into t1 values(0.00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000); +select * from t1; +col1 +0.000000000000000000000000000000 +select 0.00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000; +0.00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 +0.000000000000000000000000000000000000000000000000000000000000000000000000 +select 0.0000000000000000000000000000000000000000000000000000000000000000000000012; +0.0000000000000000000000000000000000000000000000000000000000000000000000012 +0.000000000000000000000000000000000000000000000000000000000000000000000001 +select 0.000000000000000000000000000000000000000000000000000000000000000000000001; +0.000000000000000000000000000000000000000000000000000000000000000000000001 +0.000000000000000000000000000000000000000000000000000000000000000000000001 +drop table if exists dd; +CREATE TABLE dd(a date, b datetime, c timestamp); +SET sql_mode=''; +TRUNCATE TABLE dd; +INSERT INTO dd(a) values('0000-00-00'); +SHOW WARNINGS; +Level Code Message +SELECT a FROM dd; +a +0000-00-00 +TRUNCATE TABLE dd; +INSERT INTO dd(b) values('2000-10-01'); +UPDATE dd SET b = '0000-00-00'; +SHOW WARNINGS; +Level Code Message +SELECT b FROM dd; +b +0000-00-00 00:00:00 +TRUNCATE TABLE dd; +INSERT INTO dd(c) values('0000-00-00 20:00:00'); +SHOW WARNINGS; +Level Code Message +Warning 1292 Incorrect timestamp value: '0000-00-00 20:00:00' for column 'c' at row 1 +SELECT c FROM dd; +c +0000-00-00 00:00:00 +TRUNCATE TABLE dd; +INSERT INTO dd(c) values('2000-10-01 20:00:00'); +UPDATE dd SET c = '0000-00-00 20:00:00'; +SHOW WARNINGS; +Level Code Message +Warning 1292 Incorrect timestamp value: '0000-00-00 20:00:00' +SELECT c FROM dd; +c +0000-00-00 00:00:00 +SET sql_mode='NO_ZERO_DATE'; +TRUNCATE TABLE dd; +INSERT INTO dd(b) values('0000-0-00'); +SHOW WARNINGS; +Level Code Message +Warning 1292 Incorrect datetime value: '0000-0-00' for column 'b' at row 1 +SELECT b FROM dd; +b +0000-00-00 00:00:00 +TRUNCATE TABLE dd; +INSERT INTO dd(a) values('2000-10-01'); +UPDATE dd SET a = '0000-00-00'; +SHOW WARNINGS; +Level Code Message +Warning 1292 Incorrect date value: '0000-00-00' +SELECT a FROM dd; +a +0000-00-00 +TRUNCATE TABLE dd; +INSERT INTO dd(c) values('2000-10-01 10:00:00'); +UPDATE dd SET c = '0000-00-00 10:00:00'; +SHOW WARNINGS; +Level Code Message +Warning 1292 Incorrect timestamp value: '0000-00-00 10:00:00' +SELECT c FROM dd; +c +0000-00-00 00:00:00 +SET sql_mode='NO_ZERO_DATE,STRICT_TRANS_TABLES'; +TRUNCATE TABLE dd; +INSERT INTO dd(c) VALUES ('0000-00-00 20:00:00'); +Error 1292 (22007): Incorrect timestamp value: '0000-00-00 20:00:00' for column 'c' at row 1 +INSERT IGNORE INTO dd(c) VALUES ('0000-00-00 20:00:00'); +SHOW WARNINGS; +Level Code Message +Warning 1292 Incorrect timestamp value: '0000-00-00 20:00:00' for column 'c' at row 1 +SELECT c FROM dd; +c +0000-00-00 00:00:00 +TRUNCATE TABLE dd; +INSERT INTO dd(b) values('2000-10-01'); +UPDATE dd SET b = '0000-00-00'; +Error 1292 (22007): Incorrect datetime value: '0000-00-00' +UPDATE IGNORE dd SET b = '0000-00-00'; +SHOW WARNINGS; +Level Code Message +Warning 1292 Incorrect datetime value: '0000-00-00' +SELECT b FROM dd; +b +0000-00-00 00:00:00 +TRUNCATE TABLE dd; +INSERT INTO dd(c) values('2000-10-01 10:00:00'); +UPDATE dd SET c = '0000-00-00 00:00:00'; +Error 1292 (22007): Incorrect timestamp value: '0000-00-00 00:00:00' +UPDATE IGNORE dd SET c = '0000-00-00 00:00:00'; +SHOW WARNINGS; +Level Code Message +Warning 1292 Incorrect timestamp value: '0000-00-00 00:00:00' +SELECT c FROM dd; +c +0000-00-00 00:00:00 +SET sql_mode=''; +TRUNCATE TABLE dd; +INSERT INTO dd(a) values('2000-01-00'); +SHOW WARNINGS; +Level Code Message +SELECT a FROM dd; +a +2000-01-00 +INSERT INTO dd(a) values('2000-00-01'); +SHOW WARNINGS; +Level Code Message +SELECT a FROM dd; +a +2000-01-00 +2000-00-01 +INSERT INTO dd(a) values('0-01-02'); +SHOW WARNINGS; +Level Code Message +SELECT a FROM dd; +a +2000-01-00 +2000-00-01 +2000-01-02 +TRUNCATE TABLE dd; +INSERT INTO dd(b) values('2000-01-02'); +UPDATE dd SET b = '2000-00-02'; +SHOW WARNINGS; +Level Code Message +SELECT b FROM dd; +b +2000-00-02 00:00:00 +TRUNCATE TABLE dd; +INSERT INTO dd(c) values('2000-01-02 20:00:00'); +UPDATE dd SET c = '0000-01-02 20:00:00'; +SHOW WARNINGS; +Level Code Message +Warning 1292 Incorrect timestamp value: '0000-01-02 20:00:00' +SELECT c FROM dd; +c +0000-00-00 00:00:00 +SET sql_mode='NO_ZERO_IN_DATE'; +TRUNCATE TABLE dd; +INSERT INTO dd(a) values('2000-01-00'); +SHOW WARNINGS; +Level Code Message +Warning 1292 Incorrect date value: '2000-01-00' for column 'a' at row 1 +SELECT a FROM dd; +a +0000-00-00 +TRUNCATE TABLE dd; +INSERT INTO dd(a) values('2000-01-02'); +UPDATE dd SET a = '2000-00-02'; +SHOW WARNINGS; +Level Code Message +Warning 1292 Incorrect date value: '2000-00-02' +SELECT a FROM dd; +a +0000-00-00 +UPDATE dd SET b = '2000-01-0'; +SHOW WARNINGS; +Level Code Message +Warning 1292 Incorrect datetime value: '2000-01-0' +SELECT b FROM dd; +b +0000-00-00 00:00:00 +UPDATE dd SET b = '0-01-02'; +SHOW WARNINGS; +Level Code Message +SELECT b FROM dd; +b +2000-01-02 00:00:00 +TRUNCATE TABLE dd; +INSERT INTO dd(c) values('2000-01-02 20:00:00'); +UPDATE dd SET c = '2000-00-02 20:00:00'; +SHOW WARNINGS; +Level Code Message +Warning 1292 Incorrect timestamp value: '2000-00-02 20:00:00' +SELECT c FROM dd; +c +0000-00-00 00:00:00 +SET sql_mode='NO_ZERO_IN_DATE,STRICT_TRANS_TABLES'; +TRUNCATE TABLE dd; +INSERT INTO dd(b) VALUES ('2000-01-00'); +Error 1292 (22007): Incorrect datetime value: '2000-01-00' for column 'b' at row 1 +INSERT IGNORE INTO dd(b) VALUES ('2000-00-01'); +SHOW WARNINGS; +Level Code Message +Warning 1292 Incorrect datetime value: '2000-00-01' for column 'b' at row 1 +SELECT b FROM dd; +b +0000-00-00 00:00:00 +TRUNCATE TABLE dd; +INSERT INTO dd(b) VALUES ('2000-01-02'); +UPDATE dd SET b = '2000-01-00'; +Error 1292 (22007): Incorrect datetime value: '2000-01-00' +UPDATE IGNORE dd SET b = '2000-01-0'; +SHOW WARNINGS; +Level Code Message +Warning 1292 Incorrect datetime value: '2000-01-0' +SELECT b FROM dd; +b +0000-00-00 00:00:00 +UPDATE dd SET b = '0000-1-2'; +SELECT b FROM dd; +b +0000-01-02 00:00:00 +UPDATE dd SET c = '0000-01-05'; +Error 1292 (22007): Incorrect timestamp value: '0000-01-05' +UPDATE IGNORE dd SET c = '0000-01-5'; +SHOW WARNINGS; +Level Code Message +Warning 1292 Incorrect timestamp value: '0000-01-5' +SELECT c FROM dd; +c +0000-00-00 00:00:00 +TRUNCATE TABLE dd; +INSERT INTO dd(c) VALUES ('2000-01-00 20:00:00'); +Error 1292 (22007): Incorrect timestamp value: '2000-01-00 20:00:00' for column 'c' at row 1 +INSERT INTO dd(c) VALUES ('2000-01-02'); +UPDATE dd SET c = '2000-01-00 20:00:00'; +Error 1292 (22007): Incorrect timestamp value: '2000-01-00 20:00:00' +UPDATE IGNORE dd SET b = '2000-01-00'; +SHOW WARNINGS; +Level Code Message +Warning 1292 Incorrect datetime value: '2000-01-00' +SELECT b FROM dd; +b +0000-00-00 00:00:00 +SET sql_mode='ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION'; +TRUNCATE TABLE dd; +INSERT INTO dd(a) values('0000-00-00'); +SHOW WARNINGS; +Level Code Message +SELECT a FROM dd; +a +0000-00-00 +TRUNCATE TABLE dd; +INSERT INTO dd(b) values('2000-10-01'); +UPDATE dd SET b = '0000-00-00'; +SHOW WARNINGS; +Level Code Message +SELECT b FROM dd; +b +0000-00-00 00:00:00 +TRUNCATE TABLE dd; +INSERT INTO dd(c) values('0000-00-00 00:00:00'); +SHOW WARNINGS; +Level Code Message +TRUNCATE TABLE dd; +INSERT INTO dd(c) values('2000-10-01 10:00:00'); +UPDATE dd SET c = '0000-00-00 00:00:00'; +SHOW WARNINGS; +Level Code Message +SELECT c FROM dd; +c +0000-00-00 00:00:00 +TRUNCATE TABLE dd; +INSERT INTO dd(b) VALUES ('2000-01-00'); +Error 1292 (22007): Incorrect datetime value: '2000-01-00' for column 'b' at row 1 +INSERT IGNORE INTO dd(b) VALUES ('2000-00-01'); +SHOW WARNINGS; +Level Code Message +Warning 1292 Incorrect datetime value: '2000-00-01' for column 'b' at row 1 +SELECT b FROM dd; +b +0000-00-00 00:00:00 +TRUNCATE TABLE dd; +INSERT INTO dd(b) VALUES ('2000-01-02'); +UPDATE dd SET b = '2000-01-00'; +Error 1292 (22007): Incorrect datetime value: '2000-01-00' +UPDATE IGNORE dd SET b = '2000-01-0'; +SHOW WARNINGS; +Level Code Message +Warning 1292 Incorrect datetime value: '2000-01-0' +SELECT b FROM dd; +b +0000-00-00 00:00:00 +UPDATE dd SET b = '0000-1-2'; +SELECT b FROM dd; +b +0000-01-02 00:00:00 +UPDATE dd SET c = '0000-01-05'; +Error 1292 (22007): Incorrect timestamp value: '0000-01-05' +UPDATE IGNORE dd SET c = '0000-01-5'; +SHOW WARNINGS; +Level Code Message +Warning 1292 Incorrect timestamp value: '0000-01-5' +SELECT c FROM dd; +c +0000-00-00 00:00:00 +TRUNCATE TABLE dd; +INSERT INTO dd(c) VALUES ('2000-01-00 20:00:00'); +Error 1292 (22007): Incorrect timestamp value: '2000-01-00 20:00:00' for column 'c' at row 1 +INSERT INTO dd(c) VALUES ('2000-01-02'); +UPDATE dd SET c = '2000-01-00 20:00:00'; +Error 1292 (22007): Incorrect timestamp value: '2000-01-00 20:00:00' +UPDATE IGNORE dd SET b = '2000-01-00'; +SHOW WARNINGS; +Level Code Message +Warning 1292 Incorrect datetime value: '2000-01-00' +SELECT b FROM dd; +b +0000-00-00 00:00:00 +SET sql_mode='ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION'; +TRUNCATE TABLE dd; +INSERT INTO dd(a) values('2000-00-10'); +SHOW WARNINGS; +Level Code Message +SELECT a FROM dd; +a +2000-00-10 +TRUNCATE TABLE dd; +INSERT INTO dd(b) values('2000-10-01'); +UPDATE dd SET b = '2000-00-10'; +SHOW WARNINGS; +Level Code Message +SELECT b FROM dd; +b +2000-00-10 00:00:00 +TRUNCATE TABLE dd; +INSERT INTO dd(c) values('2000-10-01 10:00:00'); +UPDATE dd SET c = '2000-00-10 00:00:00'; +Error 1292 (22007): Incorrect timestamp value: '2000-00-10 00:00:00' +UPDATE IGNORE dd SET c = '2000-01-00 00:00:00'; +SHOW WARNINGS; +Level Code Message +Warning 1292 Incorrect timestamp value: '2000-01-00 00:00:00' +SELECT c FROM dd; +c +0000-00-00 00:00:00 +drop table if exists table_20220419; +CREATE TABLE table_20220419 ( +id bigint(20) NOT NULL AUTO_INCREMENT, +lastLoginDate datetime NOT NULL, +PRIMARY KEY (id) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin; +set sql_mode=''; +insert into table_20220419 values(1,'0000-00-00 00:00:00'); +set sql_mode='ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_ENGINE_SUBSTITUTION'; +insert into table_20220419(lastLoginDate) select lastLoginDate from table_20220419; +Error 1292 (22007): Incorrect datetime value: '0000-00-00 00:00:00' +set sql_mode=default; diff --git a/tests/integrationtest/r/planner/core/issuetest/planner_issue.result b/tests/integrationtest/r/planner/core/issuetest/planner_issue.result new file mode 100644 index 0000000000000..265eb9f1aba8a --- /dev/null +++ b/tests/integrationtest/r/planner/core/issuetest/planner_issue.result @@ -0,0 +1,315 @@ +CREATE TABLE aa311c3c ( +57fd8d09 year(4) DEFAULT '1913', +afbdd7c3 char(220) DEFAULT 'gakkl6occ0yd2jmhi2qxog8szibtcqwxyxmga3hp4ktszjplmg3rjvu8v6lgn9q6hva2lekhw6napjejbut6svsr8q2j8w8rc551e5vq', +43b06e99 date NOT NULL DEFAULT '3403-10-08', +b80b3746 tinyint(4) NOT NULL DEFAULT '34', +6302d8ac timestamp DEFAULT '2004-04-01 18:21:18', +PRIMARY KEY (43b06e99,b80b3746) /*T![clustered_index] CLUSTERED */, +KEY 3080c821 (57fd8d09,43b06e99,b80b3746), +KEY a9af33a4 (57fd8d09,b80b3746,43b06e99), +KEY 464b386e (b80b3746), +KEY 19dc3c2d (57fd8d09) +) ENGINE=InnoDB DEFAULT CHARSET=ascii COLLATE=ascii_bin COMMENT='320f8401'; +explain select /*+ use_index_merge( `aa311c3c` ) */ `aa311c3c`.`43b06e99` as r0 , `aa311c3c`.`6302d8ac` as r1 from `aa311c3c` where IsNull( `aa311c3c`.`b80b3746` ) or not( `aa311c3c`.`57fd8d09` >= '2008' ) order by r0,r1 limit 95; +id estRows task access object operator info +Projection_7 95.00 root planner__core__issuetest__planner_issue.aa311c3c.43b06e99, planner__core__issuetest__planner_issue.aa311c3c.6302d8ac +└─TopN_9 95.00 root planner__core__issuetest__planner_issue.aa311c3c.43b06e99, planner__core__issuetest__planner_issue.aa311c3c.6302d8ac, offset:0, count:95 + └─IndexMerge_17 95.00 root type: union + ├─TableFullScan_13(Build) 0.00 cop[tikv] table:aa311c3c keep order:false, stats:pseudo + ├─IndexRangeScan_14(Build) 3323.33 cop[tikv] table:aa311c3c, index:3080c821(57fd8d09, 43b06e99, b80b3746) range:[-inf,2008), keep order:false, stats:pseudo + └─TopN_16(Probe) 95.00 cop[tikv] planner__core__issuetest__planner_issue.aa311c3c.43b06e99, planner__core__issuetest__planner_issue.aa311c3c.6302d8ac, offset:0, count:95 + └─TableRowIDScan_15 3323.33 cop[tikv] table:aa311c3c keep order:false, stats:pseudo +CREATE TABLE t1(id int,col1 varchar(10),col2 varchar(10),col3 varchar(10)); +CREATE TABLE t2(id int,col1 varchar(10),col2 varchar(10),col3 varchar(10)); +INSERT INTO t1 values(1,NULL,NULL,null),(2,NULL,NULL,null),(3,NULL,NULL,null); +INSERT INTO t2 values(1,'a','aa','aaa'),(2,'b','bb','bbb'),(3,'c','cc','ccc'); +WITH tmp AS (SELECT t2.* FROM t2) select (SELECT tmp.col1 FROM tmp WHERE tmp.id=t1.id ) col1, (SELECT tmp.col2 FROM tmp WHERE tmp.id=t1.id ) col2, (SELECT tmp.col3 FROM tmp WHERE tmp.id=t1.id ) col3 from t1; +col1 col2 col3 +a aa aaa +b bb bbb +c cc ccc +set tidb_enable_index_merge=on; +drop table if exists t; +create table t(a int, b int, index idx_a(a), index idx_b(b)); +set @@session.sql_select_limit=3; +explain format = 'brief' select * from t where a = 1 or b = 1; +id estRows task access object operator info +IndexMerge 3.00 root type: union, limit embedded(offset:0, count:3) +├─Limit(Build) 1.50 cop[tikv] offset:0, count:3 +│ └─IndexRangeScan 1.50 cop[tikv] table:t, index:idx_a(a) range:[1,1], keep order:false, stats:pseudo +├─Limit(Build) 1.50 cop[tikv] offset:0, count:3 +│ └─IndexRangeScan 1.50 cop[tikv] table:t, index:idx_b(b) range:[1,1], keep order:false, stats:pseudo +└─TableRowIDScan(Probe) 3.00 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select /*+ use_index_merge(t) */ * from t where a = 1 or b = 1; +id estRows task access object operator info +IndexMerge 3.00 root type: union, limit embedded(offset:0, count:3) +├─Limit(Build) 1.50 cop[tikv] offset:0, count:3 +│ └─IndexRangeScan 1.50 cop[tikv] table:t, index:idx_a(a) range:[1,1], keep order:false, stats:pseudo +├─Limit(Build) 1.50 cop[tikv] offset:0, count:3 +│ └─IndexRangeScan 1.50 cop[tikv] table:t, index:idx_b(b) range:[1,1], keep order:false, stats:pseudo +└─TableRowIDScan(Probe) 3.00 cop[tikv] table:t keep order:false, stats:pseudo +set @@session.sql_select_limit=18446744073709551615; +explain format = 'brief' select * from t where a = 1 or b = 1; +id estRows task access object operator info +IndexMerge 19.99 root type: union +├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx_a(a) range:[1,1], keep order:false, stats:pseudo +├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx_b(b) range:[1,1], keep order:false, stats:pseudo +└─TableRowIDScan(Probe) 19.99 cop[tikv] table:t keep order:false, stats:pseudo +explain format = 'brief' select * from t where a = 1 or b = 1 limit 3; +id estRows task access object operator info +IndexMerge 3.00 root type: union, limit embedded(offset:0, count:3) +├─Limit(Build) 1.50 cop[tikv] offset:0, count:3 +│ └─IndexRangeScan 1.50 cop[tikv] table:t, index:idx_a(a) range:[1,1], keep order:false, stats:pseudo +├─Limit(Build) 1.50 cop[tikv] offset:0, count:3 +│ └─IndexRangeScan 1.50 cop[tikv] table:t, index:idx_b(b) range:[1,1], keep order:false, stats:pseudo +└─TableRowIDScan(Probe) 3.00 cop[tikv] table:t keep order:false, stats:pseudo +drop table if exists t1, t2; +CREATE TABLE t1(id int,col1 varchar(10),col2 varchar(10),col3 varchar(10)); +CREATE TABLE t2(id int,col1 varchar(10),col2 varchar(10),col3 varchar(10)); +INSERT INTO t1 values(1,NULL,NULL,null),(2,NULL,NULL,null),(3,NULL,NULL,null); +INSERT INTO t2 values(1,'a','aa','aaa'),(2,'b','bb','bbb'),(3,'c','cc','ccc'); +WITH tmp AS (SELECT t2.* FROM t2) SELECT * FROM t1 WHERE t1.id = (select id from tmp where id = 1) or t1.id = (select id from tmp where id = 2) or t1.id = (select id from tmp where id = 3); +id col1 col2 col3 +1 NULL NULL NULL +2 NULL NULL NULL +3 NULL NULL NULL +drop table if exists t1, t2; +CREATE TABLE t1 (a INT, b INT); +CREATE TABLE t2 (a INT, b INT); +INSERT INTO t1 VALUES (1, 1); +INSERT INTO t2 VALUES (1, 1); +SELECT one.a, one.b as b2 FROM t1 one ORDER BY (SELECT two.b FROM t2 two WHERE two.a = one.b); +a b2 +1 1 +CREATE TABLE ads_txn ( +`cusno` varchar(10) NOT NULL, +`txn_dt` varchar(8) NOT NULL, +`unn_trno` decimal(22,0) NOT NULL, +`aml_cntpr_accno` varchar(64) DEFAULT NULL, +`acpayr_accno` varchar(35) DEFAULT NULL, +PRIMARY KEY (`cusno`,`txn_dt`,`unn_trno`) NONCLUSTERED +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY LIST COLUMNS(`txn_dt`) +(PARTITION `p20000101` VALUES IN ('20000101'), +PARTITION `p20220101` VALUES IN ('20220101'), +PARTITION `p20230516` VALUES IN ('20230516')); +analyze table ads_txn; +set autocommit=OFF; +explain update ads_txn s set aml_cntpr_accno = trim(acpayr_accno) where s._tidb_rowid between 1 and 100000; +id estRows task access object operator info +Update_5 N/A root N/A +└─Projection_6 8000.00 root planner__core__issuetest__planner_issue.ads_txn.cusno, planner__core__issuetest__planner_issue.ads_txn.txn_dt, planner__core__issuetest__planner_issue.ads_txn.unn_trno, planner__core__issuetest__planner_issue.ads_txn.aml_cntpr_accno, planner__core__issuetest__planner_issue.ads_txn.acpayr_accno, planner__core__issuetest__planner_issue.ads_txn._tidb_rowid + └─SelectLock_7 8000.00 root for update 0 + └─TableReader_9 10000.00 root partition:all data:TableRangeScan_8 + └─TableRangeScan_8 10000.00 cop[tikv] table:s range:[1,100000], keep order:false, stats:pseudo +CREATE TABLE tb1 (cid INT, code INT, class VARCHAR(10)); +CREATE TABLE tb2 (cid INT, code INT, class VARCHAR(10)); +UPDATE tb1, (SELECT code AS cid, code, MAX(class) AS class FROM tb2 GROUP BY code) tb3 SET tb1.cid = tb3.cid, tb1.code = tb3.code, tb1.class = tb3.class; +CREATE TEMPORARY TABLE v0(v1 int); +INSERT INTO v0 WITH ta2 AS (TABLE v0) TABLE ta2 FOR UPDATE OF ta2; +create table tbl_39(col_239 year(4) not null default '2009', primary key(col_239), unique key idx_223(col_239), key idx_224(col_239)); +insert into tbl_39 values (1994),(1995),(1996),(1997); +explain select /*+ use_index_merge( tbl_39) */ col_239 from tbl_39 where not( tbl_39.col_239 not in ( '1994' ) ) and tbl_39.col_239 not in ( '2004' , '2010' , '2010' ) or not( tbl_39.col_239 <= '1996' ) and not( tbl_39.col_239 between '2026' and '2011' ) order by tbl_39.col_239 limit 382; +id estRows task access object operator info +Projection_8 382.00 root planner__core__issuetest__planner_issue.tbl_39.col_239 +└─Limit_15 382.00 root offset:0, count:382 + └─UnionScan_26 382.00 root or(and(not(not(eq(planner__core__issuetest__planner_issue.tbl_39.col_239, 1994))), not(in(planner__core__issuetest__planner_issue.tbl_39.col_239, 2004, 2010, 2010))), and(not(le(planner__core__issuetest__planner_issue.tbl_39.col_239, 1996)), not(and(ge(cast(planner__core__issuetest__planner_issue.tbl_39.col_239, double UNSIGNED BINARY), 2026), le(cast(planner__core__issuetest__planner_issue.tbl_39.col_239, double UNSIGNED BINARY), 2011))))) + └─IndexReader_29 382.00 root index:Selection_28 + └─Selection_28 382.00 cop[tikv] or(and(eq(planner__core__issuetest__planner_issue.tbl_39.col_239, 1994), not(in(planner__core__issuetest__planner_issue.tbl_39.col_239, 2004, 2010, 2010))), and(gt(planner__core__issuetest__planner_issue.tbl_39.col_239, 1996), or(lt(cast(planner__core__issuetest__planner_issue.tbl_39.col_239, double UNSIGNED BINARY), 2026), gt(cast(planner__core__issuetest__planner_issue.tbl_39.col_239, double UNSIGNED BINARY), 2011)))) + └─IndexRangeScan_27 477.50 cop[tikv] table:tbl_39, index:PRIMARY(col_239) range:[1994,1994], (1996,+inf], keep order:true, stats:pseudo +select /*+ use_index_merge( tbl_39) */ col_239 from tbl_39 where not( tbl_39.col_239 not in ( '1994' ) ) and tbl_39.col_239 not in ( '2004' , '2010' , '2010' ) or not( tbl_39.col_239 <= '1996' ) and not( tbl_39.col_239 between '2026' and '2011' ) order by tbl_39.col_239 limit 382; +col_239 +1994 +1997 +drop table if exists t, t1, t2; +create table t (id int,name varchar(10)); +insert into t values(1,'tt'); +create table t1(id int,name varchar(10),name1 varchar(10),name2 varchar(10)); +insert into t1 values(1,'tt','ttt','tttt'),(2,'dd','ddd','dddd'); +create table t2(id int,name varchar(10),name1 varchar(10),name2 varchar(10),`date1` date); +insert into t2 values(1,'tt','ttt','tttt','2099-12-31'),(2,'dd','ddd','dddd','2099-12-31'); +WITH bzzs AS ( +SELECT +count(1) AS bzn +FROM +t c +), +tmp1 AS ( +SELECT +t1.* +FROM +t1 +LEFT JOIN bzzs ON 1 = 1 +WHERE +name IN ('tt') +AND bzn <> 1 +), +tmp2 AS ( +SELECT +tmp1.*, +date('2099-12-31') AS endate +FROM +tmp1 +), +tmp3 AS ( +SELECT +* +FROM +tmp2 +WHERE +endate > CURRENT_DATE +UNION ALL +SELECT +'1' AS id, +'ss' AS name, +'sss' AS name1, +'ssss' AS name2, +date('2099-12-31') AS endate +FROM +bzzs t1 +WHERE +bzn = 1 +) +SELECT +c2.id, +c3.id +FROM +t2 db +LEFT JOIN tmp3 c2 ON c2.id = '1' +LEFT JOIN tmp3 c3 ON c3.id = '1'; +id id +1 1 +1 1 +drop table if exists t; +create table t(a int, b int); +set @@tidb_max_chunk_size = 32; +insert into t values(1, 1); +insert into t select a+1, a+1 from t; +insert into t select a+2, a+2 from t; +insert into t select a+4, a+4 from t; +insert into t select a+8, a+8 from t; +insert into t select a+16, a+16 from t; +insert into t select a+32, a+32 from t; +select a from (select 100 as a, 100 as b union all select * from t) t where b != 0; +a +100 +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 +15 +16 +17 +18 +19 +20 +21 +22 +23 +24 +25 +26 +27 +28 +29 +30 +31 +32 +33 +34 +35 +36 +37 +38 +39 +40 +41 +42 +43 +44 +45 +46 +47 +48 +49 +50 +51 +52 +53 +54 +55 +56 +57 +58 +59 +60 +61 +62 +63 +64 +set @@tidb_max_chunk_size = default; +drop table if exists t1, t2; +create table t1(a varchar(20) collate utf8mb4_bin, index ia(a)); +insert into t1 value('测试'),('测试 '),('xxx '); +explain format = brief select *,length(a) from t1 where a like '测试 %'; +id estRows task access object operator info +Projection 250.00 root planner__core__issuetest__planner_issue.t1.a, length(planner__core__issuetest__planner_issue.t1.a)->Column#3 +└─UnionScan 250.00 root like(planner__core__issuetest__planner_issue.t1.a, "测试 %", 92) + └─IndexReader 250.00 root index:Selection + └─Selection 250.00 cop[tikv] like(planner__core__issuetest__planner_issue.t1.a, "测试 %", 92) + └─IndexRangeScan 250.00 cop[tikv] table:t1, index:ia(a) range:["测试","测试!"), keep order:false, stats:pseudo +explain format = brief select *,length(a) from t1 where a like '测试'; +id estRows task access object operator info +Projection 10.00 root planner__core__issuetest__planner_issue.t1.a, length(planner__core__issuetest__planner_issue.t1.a)->Column#3 +└─UnionScan 10.00 root like(planner__core__issuetest__planner_issue.t1.a, "测试", 92) + └─IndexReader 10.00 root index:Selection + └─Selection 10.00 cop[tikv] like(planner__core__issuetest__planner_issue.t1.a, "测试", 92) + └─IndexRangeScan 10.00 cop[tikv] table:t1, index:ia(a) range:["测试","测试"], keep order:false, stats:pseudo +select *,length(a) from t1 where a like '测试 %'; +a length(a) +测试 8 +select *,length(a) from t1 where a like '测试'; +a length(a) +测试 6 +explain format = brief select * from t1 use index (ia) where a like 'xxx_'; +id estRows task access object operator info +Projection 250.00 root planner__core__issuetest__planner_issue.t1.a +└─UnionScan 250.00 root like(planner__core__issuetest__planner_issue.t1.a, "xxx_", 92) + └─IndexReader 250.00 root index:Selection + └─Selection 250.00 cop[tikv] like(planner__core__issuetest__planner_issue.t1.a, "xxx_", 92) + └─IndexRangeScan 250.00 cop[tikv] table:t1, index:ia(a) range:["xxx","xxy"), keep order:false, stats:pseudo +select * from t1 use index (ia) where a like 'xxx_'; +a +xxx +create table t2(a varchar(20) collate gbk_chinese_ci, index ia(a)); +insert into t2 value('测试'),('测试 '); +explain format = brief select *,length(a) from t2 where a like '测试 %'; +id estRows task access object operator info +Projection 250.00 root planner__core__issuetest__planner_issue.t2.a, length(to_binary(planner__core__issuetest__planner_issue.t2.a))->Column#3 +└─UnionScan 250.00 root like(planner__core__issuetest__planner_issue.t2.a, "测试 %", 92) + └─IndexReader 250.00 root index:Selection + └─Selection 250.00 cop[tikv] like(planner__core__issuetest__planner_issue.t2.a, "测试 %", 92) + └─IndexRangeScan 250.00 cop[tikv] table:t2, index:ia(a) range:["\x89\a\xba%","\x89\a\xba%!"), keep order:false, stats:pseudo +explain format = brief select *,length(a) from t2 where a like '测试'; +id estRows task access object operator info +Projection 10.00 root planner__core__issuetest__planner_issue.t2.a, length(to_binary(planner__core__issuetest__planner_issue.t2.a))->Column#3 +└─UnionScan 10.00 root like(planner__core__issuetest__planner_issue.t2.a, "测试", 92) + └─IndexReader 10.00 root index:Selection + └─Selection 10.00 cop[tikv] like(planner__core__issuetest__planner_issue.t2.a, "测试", 92) + └─IndexRangeScan 10.00 cop[tikv] table:t2, index:ia(a) range:["\x89\a\xba%","\x89\a\xba%"], keep order:false, stats:pseudo +select *,length(a) from t2 where a like '测试 %'; +a length(a) +测试 6 +select *,length(a) from t2 where a like '测试'; +a length(a) +测试 4 diff --git a/tests/integrationtest/r/planner/core/range_scan_for_like.result b/tests/integrationtest/r/planner/core/range_scan_for_like.result new file mode 100644 index 0000000000000..058de729e4589 --- /dev/null +++ b/tests/integrationtest/r/planner/core/range_scan_for_like.result @@ -0,0 +1,1035 @@ +create table t(a varchar(20) collate utf8mb4_general_ci, index ia(a)); +insert into t value('测试'),('测试Abc'),('测试 '),('你好'),('aABBccdd'),('Aa'),(''),(' '),(' '),(' 语言'),(' 语 言 '),('测测试 '),('测测试 '),(NULL); +explain select *, length(a) from t use index (ia) where a like '测试%' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Projection_13 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_16 250.00 root index:Selection_15 + └─Selection_15 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "测试%", 92) + └─IndexRangeScan_14 250.00 cop[tikv] table:t, index:ia(a) range:["mK\x8b\xd5","mK\x8b\xd6"), keep order:true, stats:pseudo +select *, length(a) from t use index (ia) where a like '测试%' order by a,_tidb_rowid; +a length(a) +测试 6 +测试 11 +测试Abc 9 +explain select *, length(a) from t use index (ia) where a like '测%%' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Projection_13 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_16 250.00 root index:Selection_15 + └─Selection_15 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "测%%", 92) + └─IndexRangeScan_14 250.00 cop[tikv] table:t, index:ia(a) range:["mK","mL"), keep order:true, stats:pseudo +select *, length(a) from t use index (ia) where a like '测%%' order by a,_tidb_rowid; +a length(a) +测测试 10 +测测试 13 +测试 6 +测试 11 +测试Abc 9 +explain select *, length(a) from t use index (ia) where a like '测%%试' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Projection_13 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_16 250.00 root index:Selection_15 + └─Selection_15 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "测%%试", 92) + └─IndexRangeScan_14 250.00 cop[tikv] table:t, index:ia(a) range:["mK","mL"), keep order:true, stats:pseudo +select *, length(a) from t use index (ia) where a like '测%%试' order by a,_tidb_rowid; +a length(a) +测试 6 +explain select *, length(a) from t use index (ia) where a like '测试%%' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Projection_13 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_16 250.00 root index:Selection_15 + └─Selection_15 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "测试%%", 92) + └─IndexRangeScan_14 250.00 cop[tikv] table:t, index:ia(a) range:["mK\x8b\xd5","mK\x8b\xd6"), keep order:true, stats:pseudo +select *, length(a) from t use index (ia) where a like '测试%%' order by a,_tidb_rowid; +a length(a) +测试 6 +测试 11 +测试Abc 9 +explain select *, length(a) from t use index (ia) where a like '测试_' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Projection_13 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_16 250.00 root index:Selection_15 + └─Selection_15 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "测试_", 92) + └─IndexRangeScan_14 250.00 cop[tikv] table:t, index:ia(a) range:["mK\x8b\xd5","mK\x8b\xd6"), keep order:true, stats:pseudo +select *, length(a) from t use index (ia) where a like '测试_' order by a,_tidb_rowid; +a length(a) +explain select *, length(a) from t use index (ia) where a like '你好%' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Projection_13 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_16 250.00 root index:Selection_15 + └─Selection_15 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "你好%", 92) + └─IndexRangeScan_14 250.00 cop[tikv] table:t, index:ia(a) range:["O`Y}","O`Y~"), keep order:true, stats:pseudo +select *, length(a) from t use index (ia) where a like '你好%' order by a,_tidb_rowid; +a length(a) +你好 6 +explain select *, length(a) from t use index (ia) where a like 'aa' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 10.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Projection_13 10.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_16 10.00 root index:Selection_15 + └─Selection_15 10.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "aa", 92) + └─IndexRangeScan_14 10.00 cop[tikv] table:t, index:ia(a) range:["\x00A\x00A","\x00A\x00A"], keep order:true, stats:pseudo +select *, length(a) from t use index (ia) where a like 'aa' order by a,_tidb_rowid; +a length(a) +Aa 2 +explain select *, length(a) from t use index (ia) where a like 'aa%' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Projection_13 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_16 250.00 root index:Selection_15 + └─Selection_15 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "aa%", 92) + └─IndexRangeScan_14 250.00 cop[tikv] table:t, index:ia(a) range:["\x00A\x00A","\x00A\x00B"), keep order:true, stats:pseudo +select *, length(a) from t use index (ia) where a like 'aa%' order by a,_tidb_rowid; +a length(a) +Aa 2 +aABBccdd 8 +explain select *, length(a) from t use index (ia) where a like 'aa%cc' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Projection_13 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_16 250.00 root index:Selection_15 + └─Selection_15 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "aa%cc", 92) + └─IndexRangeScan_14 250.00 cop[tikv] table:t, index:ia(a) range:["\x00A\x00A","\x00A\x00B"), keep order:true, stats:pseudo +select *, length(a) from t use index (ia) where a like 'aa%cc' order by a,_tidb_rowid; +a length(a) +explain select *, length(a) from t use index (ia) where a like '' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 10.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Projection_13 10.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_16 10.00 root index:Selection_15 + └─Selection_15 10.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "", 92) + └─IndexRangeScan_14 10.00 cop[tikv] table:t, index:ia(a) range:["",""], keep order:true, stats:pseudo +select *, length(a) from t use index (ia) where a like '' order by a,_tidb_rowid; +a length(a) + 0 +explain select *, length(a) from t use index (ia) where a like ' ' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 10.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Projection_13 10.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_16 10.00 root index:Selection_15 + └─Selection_15 10.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, " ", 92) + └─IndexRangeScan_14 10.00 cop[tikv] table:t, index:ia(a) range:["",""], keep order:true, stats:pseudo +select *, length(a) from t use index (ia) where a like ' ' order by a,_tidb_rowid; +a length(a) + 1 +explain select *, length(a) from t use index (ia) where a like 'aa%dd' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Projection_13 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_16 250.00 root index:Selection_15 + └─Selection_15 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "aa%dd", 92) + └─IndexRangeScan_14 250.00 cop[tikv] table:t, index:ia(a) range:["\x00A\x00A","\x00A\x00B"), keep order:true, stats:pseudo +select *, length(a) from t use index (ia) where a like 'aa%dd' order by a,_tidb_rowid; +a length(a) +aABBccdd 8 +explain select *, length(a) from t use index (ia) where a like 'aa%%dd' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Projection_13 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_16 250.00 root index:Selection_15 + └─Selection_15 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "aa%%dd", 92) + └─IndexRangeScan_14 250.00 cop[tikv] table:t, index:ia(a) range:["\x00A\x00A","\x00A\x00B"), keep order:true, stats:pseudo +select *, length(a) from t use index (ia) where a like 'aa%%dd' order by a,_tidb_rowid; +a length(a) +aABBccdd 8 +explain select *, length(a) from t use index (ia) where a like 'aa_bccdd' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Projection_13 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_16 250.00 root index:Selection_15 + └─Selection_15 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "aa_bccdd", 92) + └─IndexRangeScan_14 250.00 cop[tikv] table:t, index:ia(a) range:["\x00A\x00A","\x00A\x00B"), keep order:true, stats:pseudo +select *, length(a) from t use index (ia) where a like 'aa_bccdd' order by a,_tidb_rowid; +a length(a) +aABBccdd 8 +explain select *, length(a) from t use index (ia) where a like '%%' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 8000.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Projection_13 8000.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_16 8000.00 root index:Selection_15 + └─Selection_15 8000.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "%%", 92) + └─IndexFullScan_14 10000.00 cop[tikv] table:t, index:ia(a) keep order:true, stats:pseudo +select *, length(a) from t use index (ia) where a like '%%' order by a,_tidb_rowid; +a length(a) + 0 + 1 + 2 + 语 言 10 + 语言 7 +Aa 2 +aABBccdd 8 +你好 6 +测测试 10 +测测试 13 +测试 6 +测试 11 +测试Abc 9 +explain select *, length(a) from t use index (ia) where a like ' %%' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Projection_13 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_16 250.00 root index:Selection_15 + └─Selection_15 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, " %%", 92) + └─IndexRangeScan_14 250.00 cop[tikv] table:t, index:ia(a) range:["","\x00!"), keep order:true, stats:pseudo +select *, length(a) from t use index (ia) where a like ' %%' order by a,_tidb_rowid; +a length(a) + 1 + 2 + 语 言 10 + 语言 7 +explain select *, length(a) from t use index (ia) where a like ' %%语言' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Projection_13 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_16 250.00 root index:Selection_15 + └─Selection_15 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, " %%语言", 92) + └─IndexRangeScan_14 250.00 cop[tikv] table:t, index:ia(a) range:["","\x00!"), keep order:true, stats:pseudo +select *, length(a) from t use index (ia) where a like ' %%语言' order by a,_tidb_rowid; +a length(a) + 语言 7 +explain select *, length(a) from t use index (ia) where a like ' 语 %' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Projection_13 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_16 250.00 root index:Selection_15 + └─Selection_15 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, " 语 %", 92) + └─IndexRangeScan_14 250.00 cop[tikv] table:t, index:ia(a) range:["\x00 \x00 \x8b\xed","\x00 \x00 \x8b\xed\x00!"), keep order:true, stats:pseudo +select *, length(a) from t use index (ia) where a like ' 语 %' order by a,_tidb_rowid; +a length(a) + 语 言 10 +explain select *, length(a) from t use index (ia) where a like ' 语 _' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Projection_13 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_16 250.00 root index:Selection_15 + └─Selection_15 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, " 语 _", 92) + └─IndexRangeScan_14 250.00 cop[tikv] table:t, index:ia(a) range:["\x00 \x00 \x8b\xed","\x00 \x00 \x8b\xed\x00 \x00!"), keep order:true, stats:pseudo +select *, length(a) from t use index (ia) where a like ' 语 _' order by a,_tidb_rowid; +a length(a) +drop table t; +create table t(a varchar(20) collate utf8mb4_unicode_ci, unique index ia(a)); +insert into t value(''),('测试'),('测试abc'),('你好'),('aabbccdd'),(' 语言'),(' 语 言 '),('测测试 '); +explain select *, length(a) from t use index (ia) where a like '测试%' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Sort_7 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─Projection_9 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_12 250.00 root index:Selection_11 + └─Selection_11 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "测试%", 92) + └─IndexRangeScan_10 250.00 cop[tikv] table:t, index:ia(a) range:["\xfb@\xedK\xfbA\x8b\xd5","\xfb@\xedK\xfbA\x8b\xd6"), keep order:false, stats:pseudo +select *, length(a) from t use index (ia) where a like '测试%' order by a,_tidb_rowid; +a length(a) +测试 6 +测试abc 9 +explain select *, length(a) from t use index (ia) where a like '测%%' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Sort_7 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─Projection_9 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_12 250.00 root index:Selection_11 + └─Selection_11 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "测%%", 92) + └─IndexRangeScan_10 250.00 cop[tikv] table:t, index:ia(a) range:["\xfb@\xedK","\xfb@\xedL"), keep order:false, stats:pseudo +select *, length(a) from t use index (ia) where a like '测%%' order by a,_tidb_rowid; +a length(a) +测测试 13 +测试 6 +测试abc 9 +explain select *, length(a) from t use index (ia) where a like '测%%试' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Sort_7 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─Projection_9 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_12 250.00 root index:Selection_11 + └─Selection_11 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "测%%试", 92) + └─IndexRangeScan_10 250.00 cop[tikv] table:t, index:ia(a) range:["\xfb@\xedK","\xfb@\xedL"), keep order:false, stats:pseudo +select *, length(a) from t use index (ia) where a like '测%%试' order by a,_tidb_rowid; +a length(a) +测试 6 +explain select *, length(a) from t use index (ia) where a like '测试%%' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Sort_7 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─Projection_9 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_12 250.00 root index:Selection_11 + └─Selection_11 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "测试%%", 92) + └─IndexRangeScan_10 250.00 cop[tikv] table:t, index:ia(a) range:["\xfb@\xedK\xfbA\x8b\xd5","\xfb@\xedK\xfbA\x8b\xd6"), keep order:false, stats:pseudo +select *, length(a) from t use index (ia) where a like '测试%%' order by a,_tidb_rowid; +a length(a) +测试 6 +测试abc 9 +explain select *, length(a) from t use index (ia) where a like '测试_' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Sort_7 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─Projection_9 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_12 250.00 root index:Selection_11 + └─Selection_11 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "测试_", 92) + └─IndexRangeScan_10 250.00 cop[tikv] table:t, index:ia(a) range:["\xfb@\xedK\xfbA\x8b\xd5","\xfb@\xedK\xfbA\x8b\xd6"), keep order:false, stats:pseudo +select *, length(a) from t use index (ia) where a like '测试_' order by a,_tidb_rowid; +a length(a) +explain select *, length(a) from t use index (ia) where a like '你好%' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Sort_7 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─Projection_9 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_12 250.00 root index:Selection_11 + └─Selection_11 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "你好%", 92) + └─IndexRangeScan_10 250.00 cop[tikv] table:t, index:ia(a) range:["\xfb@\xcf`\xfb@\xd9}","\xfb@\xcf`\xfb@\xd9~"), keep order:false, stats:pseudo +select *, length(a) from t use index (ia) where a like '你好%' order by a,_tidb_rowid; +a length(a) +你好 6 +explain select *, length(a) from t use index (ia) where a like 'aa' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 1.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Sort_7 1.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─Projection_9 1.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─Selection_11 1.00 root like(planner__core__range_scan_for_like.t.a, "aa", 92) + └─Point_Get_10 1.00 root table:t, index:ia(a) +select *, length(a) from t use index (ia) where a like 'aa' order by a,_tidb_rowid; +a length(a) +explain select *, length(a) from t use index (ia) where a like 'aa%' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Sort_7 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─Projection_9 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_12 250.00 root index:Selection_11 + └─Selection_11 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "aa%", 92) + └─IndexRangeScan_10 250.00 cop[tikv] table:t, index:ia(a) range:["\x0e3\x0e3","\x0e3\x0e4"), keep order:false, stats:pseudo +select *, length(a) from t use index (ia) where a like 'aa%' order by a,_tidb_rowid; +a length(a) +aabbccdd 8 +explain select *, length(a) from t use index (ia) where a like 'aa%cc' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Sort_7 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─Projection_9 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_12 250.00 root index:Selection_11 + └─Selection_11 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "aa%cc", 92) + └─IndexRangeScan_10 250.00 cop[tikv] table:t, index:ia(a) range:["\x0e3\x0e3","\x0e3\x0e4"), keep order:false, stats:pseudo +select *, length(a) from t use index (ia) where a like 'aa%cc' order by a,_tidb_rowid; +a length(a) +explain select *, length(a) from t use index (ia) where a like '' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 1.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Sort_7 1.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─Projection_9 1.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─Selection_11 1.00 root like(planner__core__range_scan_for_like.t.a, "", 92) + └─Point_Get_10 1.00 root table:t, index:ia(a) +select *, length(a) from t use index (ia) where a like '' order by a,_tidb_rowid; +a length(a) + 0 +explain select *, length(a) from t use index (ia) where a like 'aa%dd' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Sort_7 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─Projection_9 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_12 250.00 root index:Selection_11 + └─Selection_11 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "aa%dd", 92) + └─IndexRangeScan_10 250.00 cop[tikv] table:t, index:ia(a) range:["\x0e3\x0e3","\x0e3\x0e4"), keep order:false, stats:pseudo +select *, length(a) from t use index (ia) where a like 'aa%dd' order by a,_tidb_rowid; +a length(a) +aabbccdd 8 +explain select *, length(a) from t use index (ia) where a like 'aa%%dd' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Sort_7 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─Projection_9 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_12 250.00 root index:Selection_11 + └─Selection_11 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "aa%%dd", 92) + └─IndexRangeScan_10 250.00 cop[tikv] table:t, index:ia(a) range:["\x0e3\x0e3","\x0e3\x0e4"), keep order:false, stats:pseudo +select *, length(a) from t use index (ia) where a like 'aa%%dd' order by a,_tidb_rowid; +a length(a) +aabbccdd 8 +explain select *, length(a) from t use index (ia) where a like 'aa_bccdd' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Sort_7 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─Projection_9 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_12 250.00 root index:Selection_11 + └─Selection_11 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "aa_bccdd", 92) + └─IndexRangeScan_10 250.00 cop[tikv] table:t, index:ia(a) range:["\x0e3\x0e3","\x0e3\x0e4"), keep order:false, stats:pseudo +select *, length(a) from t use index (ia) where a like 'aa_bccdd' order by a,_tidb_rowid; +a length(a) +aabbccdd 8 +explain select *, length(a) from t use index (ia) where a like '%%' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 8000.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Sort_7 8000.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─Projection_9 8000.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_12 8000.00 root index:Selection_11 + └─Selection_11 8000.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "%%", 92) + └─IndexFullScan_10 10000.00 cop[tikv] table:t, index:ia(a) keep order:false, stats:pseudo +select *, length(a) from t use index (ia) where a like '%%' order by a,_tidb_rowid; +a length(a) + 0 + 语 言 10 + 语言 7 +aabbccdd 8 +你好 6 +测测试 13 +测试 6 +测试abc 9 +explain select *, length(a) from t use index (ia) where a like ' %%' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Sort_7 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─Projection_9 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_12 250.00 root index:Selection_11 + └─Selection_11 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, " %%", 92) + └─IndexRangeScan_10 250.00 cop[tikv] table:t, index:ia(a) range:["","\x02\n"), keep order:false, stats:pseudo +select *, length(a) from t use index (ia) where a like ' %%' order by a,_tidb_rowid; +a length(a) + 语 言 10 + 语言 7 +explain select *, length(a) from t use index (ia) where a like ' %%语言' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Sort_7 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─Projection_9 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_12 250.00 root index:Selection_11 + └─Selection_11 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, " %%语言", 92) + └─IndexRangeScan_10 250.00 cop[tikv] table:t, index:ia(a) range:["","\x02\n"), keep order:false, stats:pseudo +select *, length(a) from t use index (ia) where a like ' %%语言' order by a,_tidb_rowid; +a length(a) + 语言 7 +explain select *, length(a) from t use index (ia) where a like ' 语 %' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Sort_7 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─Projection_9 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_12 250.00 root index:Selection_11 + └─Selection_11 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, " 语 %", 92) + └─IndexRangeScan_10 250.00 cop[tikv] table:t, index:ia(a) range:["\x02\t\x02\t\xfbA\x8b\xed","\x02\t\x02\t\xfbA\x8b\xed\x02\n"), keep order:false, stats:pseudo +select *, length(a) from t use index (ia) where a like ' 语 %' order by a,_tidb_rowid; +a length(a) + 语 言 10 +explain select *, length(a) from t use index (ia) where a like ' 语 _' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, Column#3->Column#5 +└─Sort_7 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─Projection_9 250.00 root planner__core__range_scan_for_like.t.a, length(planner__core__range_scan_for_like.t.a)->Column#3, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexReader_12 250.00 root index:Selection_11 + └─Selection_11 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, " 语 _", 92) + └─IndexRangeScan_10 250.00 cop[tikv] table:t, index:ia(a) range:["\x02\t\x02\t\xfbA\x8b\xed","\x02\t\x02\t\xfbA\x8b\xed\x02\t\x02\n"), keep order:false, stats:pseudo +select *, length(a) from t use index (ia) where a like ' 语 _' order by a,_tidb_rowid; +a length(a) +drop table t; +create table t(a varchar(20) collate utf8mb4_0900_ai_ci, b varchar(20) collate ascii_bin, c bigint, primary key(a(1), b) clustered); +insert into t (a, b, c) values +('测试1', 'asdfgh', 345346), +('你好2', 'qqwweerrrr', 987765), +('こんにちは3', 'zxcvbnn', 1111111), +('안녕하세요4', 'asdfgh ', 3333333333), +('Ciao5', ' asdfgh', 444400), +('Hola6', ' asdfgh ', 6666), +('Bonjour ', '', 888888888), +('Olá8', ' ', 9999999), +('Привет9', ' ', 321321), +('Hallo10', '12345', 35678); +explain select * from t use index (primary) where a like '测试%' and b like 'asd%' order by a,b; +id estRows task access object operator info +Sort_5 6.25 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─TableReader_10 6.25 root data:Selection_9 + └─Selection_9 6.25 cop[tikv] like(planner__core__range_scan_for_like.t.a, "测试%", 92), like(planner__core__range_scan_for_like.t.b, "asd%", 92) + └─TableRangeScan_8 250.00 cop[tikv] table:t range:["\xfb@\xedK","\xfb@\xedL"), keep order:false, stats:pseudo +select * from t use index (primary) where a like '测试%' and b like 'asd%' order by a,b; +a b c +测试1 asdfgh 345346 +explain select * from t use index (primary) where a like '测试1' and b like 'asdfgh %' order by a,b; +id estRows task access object operator info +Sort_5 0.25 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─TableReader_10 0.25 root data:Selection_9 + └─Selection_9 0.25 cop[tikv] like(planner__core__range_scan_for_like.t.a, "测试1", 92), like(planner__core__range_scan_for_like.t.b, "asdfgh %", 92) + └─TableRangeScan_8 10.00 cop[tikv] table:t range:["\xfb@\xedK","\xfb@\xedK"], keep order:false, stats:pseudo +select * from t use index (primary) where a like '测试1' and b like 'asdfgh %' order by a,b; +a b c +explain select * from t use index (primary) where a like 'こんにち_' and b like 'zxc%' order by a,b; +id estRows task access object operator info +Sort_5 6.25 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─TableReader_10 6.25 root data:Selection_9 + └─Selection_9 6.25 cop[tikv] like(planner__core__range_scan_for_like.t.a, "こんにち_", 92), like(planner__core__range_scan_for_like.t.b, "zxc%", 92) + └─TableRangeScan_8 250.00 cop[tikv] table:t range:["=d","=e"), keep order:false, stats:pseudo +select * from t use index (primary) where a like 'こんにち_' and b like 'zxc%' order by a,b; +a b c +explain select * from t use index (primary) where a like '안녕하세요%' and b like 'asd%' order by a,b; +id estRows task access object operator info +Sort_5 6.25 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─TableReader_10 6.25 root data:Selection_9 + └─Selection_9 6.25 cop[tikv] like(planner__core__range_scan_for_like.t.a, "안녕하세요%", 92), like(planner__core__range_scan_for_like.t.b, "asd%", 92) + └─TableRangeScan_8 250.00 cop[tikv] table:t range:["<\x00 'aabb' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 3333.33 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 3333.33 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 3333.33 root + ├─IndexRangeScan_9(Build) 3333.33 cop[tikv] table:t, index:ia(a, b) range:["\x00A\x00A\x00B",+inf], keep order:false, stats:pseudo + └─Selection_11(Probe) 3333.33 cop[tikv] gt(planner__core__range_scan_for_like.t.a, "aabb") + └─TableRowIDScan_10 3333.33 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a > 'aabb' order by a,_tidb_rowid; +a b +aABBccdd 890 +你好 111 +测测试 2468 +测测试 99999 +测试 222 +测试 543 +测试Abc 324 +explain select * from t use index (ia) where a > 'aab' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 3333.33 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 3333.33 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 3333.33 root + ├─IndexRangeScan_9(Build) 3333.33 cop[tikv] table:t, index:ia(a, b) range:["\x00A\x00A\x00B",+inf], keep order:false, stats:pseudo + └─Selection_11(Probe) 3333.33 cop[tikv] gt(planner__core__range_scan_for_like.t.a, "aab") + └─TableRowIDScan_10 3333.33 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a > 'aab' order by a,_tidb_rowid; +a b +aabB 456 +aABBccdd 890 +你好 111 +测测试 2468 +测测试 99999 +测试 222 +测试 543 +测试Abc 324 +explain select * from t use index (ia) where a > 'aa' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 3333.33 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 3333.33 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 3333.33 root + ├─IndexRangeScan_9(Build) 3333.33 cop[tikv] table:t, index:ia(a, b) range:("\x00A\x00A",+inf], keep order:false, stats:pseudo + └─Selection_11(Probe) 3333.33 cop[tikv] gt(planner__core__range_scan_for_like.t.a, "aa") + └─TableRowIDScan_10 3333.33 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a > 'aa' order by a,_tidb_rowid; +a b +aab 456 +aabB 456 +aABBccdd 890 +你好 111 +测测试 2468 +测测试 99999 +测试 222 +测试 543 +测试Abc 324 +explain select * from t use index (ia) where a < 'aabb' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 3323.33 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 3323.33 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 3323.33 root + ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t, index:ia(a, b) range:[-inf,"\x00A\x00A\x00B"], keep order:false, stats:pseudo + └─Selection_11(Probe) 3323.33 cop[tikv] lt(planner__core__range_scan_for_like.t.a, "aabb") + └─TableRowIDScan_10 3323.33 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a < 'aabb' order by a,_tidb_rowid; +a b + 234 + 11111 + 66666 + 语 言 3579 + 语言 55555 +A 456 +Aa 456 +aab 456 +explain select * from t use index (ia) where a < 'aab' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 3323.33 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 3323.33 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 3323.33 root + ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t, index:ia(a, b) range:[-inf,"\x00A\x00A\x00B"), keep order:false, stats:pseudo + └─Selection_11(Probe) 3323.33 cop[tikv] lt(planner__core__range_scan_for_like.t.a, "aab") + └─TableRowIDScan_10 3323.33 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a < 'aab' order by a,_tidb_rowid; +a b + 234 + 11111 + 66666 + 语 言 3579 + 语言 55555 +A 456 +Aa 456 +explain select * from t use index (ia) where a < 'aa' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 3323.33 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 3323.33 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 3323.33 root + ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t, index:ia(a, b) range:[-inf,"\x00A\x00A"), keep order:false, stats:pseudo + └─Selection_11(Probe) 3323.33 cop[tikv] lt(planner__core__range_scan_for_like.t.a, "aa") + └─TableRowIDScan_10 3323.33 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a < 'aa' order by a,_tidb_rowid; +a b + 234 + 11111 + 66666 + 语 言 3579 + 语言 55555 +A 456 +explain select * from t use index (ia) where a != 'aa' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 6656.67 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 6656.67 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 6656.67 root + ├─IndexFullScan_9(Build) 10000.00 cop[tikv] table:t, index:ia(a, b) keep order:false, stats:pseudo + └─Selection_11(Probe) 6656.67 cop[tikv] ne(planner__core__range_scan_for_like.t.a, "aa") + └─TableRowIDScan_10 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a != 'aa' order by a,_tidb_rowid; +a b + 234 + 11111 + 66666 + 语 言 3579 + 语言 55555 +A 456 +aab 456 +aabB 456 +aABBccdd 890 +你好 111 +测测试 2468 +测测试 99999 +测试 222 +测试 543 +测试Abc 324 +explain select * from t use index (ia) where a != 'aaBbc' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 6656.67 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 6656.67 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 6656.67 root + ├─IndexFullScan_9(Build) 10000.00 cop[tikv] table:t, index:ia(a, b) keep order:false, stats:pseudo + └─Selection_11(Probe) 6656.67 cop[tikv] ne(planner__core__range_scan_for_like.t.a, "aaBbc") + └─TableRowIDScan_10 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a != 'aaBbc' order by a,_tidb_rowid; +a b + 234 + 11111 + 66666 + 语 言 3579 + 语言 55555 +A 456 +Aa 456 +aab 456 +aabB 456 +aABBccdd 890 +你好 111 +测测试 2468 +测测试 99999 +测试 222 +测试 543 +测试Abc 324 +explain select * from t use index (ia) where a like '测试abc' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 10.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 10.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 10.00 root + ├─IndexRangeScan_9(Build) 10.00 cop[tikv] table:t, index:ia(a, b) range:["mK\x8b\xd5\x00A","mK\x8b\xd5\x00A"], keep order:false, stats:pseudo + └─Selection_11(Probe) 10.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "测试abc", 92) + └─TableRowIDScan_10 10.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a like '测试abc' order by a,_tidb_rowid; +a b +测试Abc 324 +explain select * from t use index (ia) where a = '测试abc' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 10.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 10.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 10.00 root + ├─IndexRangeScan_9(Build) 10.00 cop[tikv] table:t, index:ia(a, b) range:["mK\x8b\xd5\x00A","mK\x8b\xd5\x00A"], keep order:false, stats:pseudo + └─Selection_11(Probe) 10.00 cop[tikv] eq(planner__core__range_scan_for_like.t.a, "测试abc") + └─TableRowIDScan_10 10.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a = '测试abc' order by a,_tidb_rowid; +a b +测试Abc 324 +explain select * from t use index (ia) where a like 'aa' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 10.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 10.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 10.00 root + ├─IndexRangeScan_9(Build) 10.00 cop[tikv] table:t, index:ia(a, b) range:["\x00A\x00A","\x00A\x00A"], keep order:false, stats:pseudo + └─Selection_11(Probe) 10.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "aa", 92) + └─TableRowIDScan_10 10.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a like 'aa' order by a,_tidb_rowid; +a b +Aa 456 +explain select * from t use index (ia) where a = 'aa' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 10.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 10.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 10.00 root + ├─IndexRangeScan_9(Build) 10.00 cop[tikv] table:t, index:ia(a, b) range:["\x00A\x00A","\x00A\x00A"], keep order:false, stats:pseudo + └─Selection_11(Probe) 10.00 cop[tikv] eq(planner__core__range_scan_for_like.t.a, "aa") + └─TableRowIDScan_10 10.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a = 'aa' order by a,_tidb_rowid; +a b +Aa 456 +explain select * from t use index (ia) where a like '测测试 ' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 10.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 10.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 10.00 root + ├─IndexRangeScan_9(Build) 10.00 cop[tikv] table:t, index:ia(a, b) range:["mKmK\x8b\xd5","mKmK\x8b\xd5"], keep order:false, stats:pseudo + └─Selection_11(Probe) 10.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "测测试 ", 92) + └─TableRowIDScan_10 10.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a like '测测试 ' order by a,_tidb_rowid; +a b +测测试 2468 +explain select * from t use index (ia) where a = '测测试 ' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 10.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 10.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 10.00 root + ├─IndexRangeScan_9(Build) 10.00 cop[tikv] table:t, index:ia(a, b) range:["mKmK\x8b\xd5","mKmK\x8b\xd5"], keep order:false, stats:pseudo + └─Selection_11(Probe) 10.00 cop[tikv] eq(planner__core__range_scan_for_like.t.a, "测测试 ") + └─TableRowIDScan_10 10.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a = '测测试 ' order by a,_tidb_rowid; +a b +测测试 2468 +测测试 99999 +explain select * from t use index (ia) where a like ' 语 言' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 10.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 10.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 10.00 root + ├─IndexRangeScan_9(Build) 10.00 cop[tikv] table:t, index:ia(a, b) range:["\x00 \x00 \x8b\xed","\x00 \x00 \x8b\xed"], keep order:false, stats:pseudo + └─Selection_11(Probe) 10.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, " 语 言", 92) + └─TableRowIDScan_10 10.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a like ' 语 言' order by a,_tidb_rowid; +a b + 语 言 3579 +explain select * from t use index (ia) where a = ' 语 言' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 10.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 10.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 10.00 root + ├─IndexRangeScan_9(Build) 10.00 cop[tikv] table:t, index:ia(a, b) range:["\x00 \x00 \x8b\xed","\x00 \x00 \x8b\xed"], keep order:false, stats:pseudo + └─Selection_11(Probe) 10.00 cop[tikv] eq(planner__core__range_scan_for_like.t.a, " 语 言") + └─TableRowIDScan_10 10.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a = ' 语 言' order by a,_tidb_rowid; +a b + 语 言 3579 +explain select * from t use index (ia) where a like '测试%' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 250.00 root + ├─IndexRangeScan_9(Build) 250.00 cop[tikv] table:t, index:ia(a, b) range:["mK\x8b\xd5","mK\x8b\xd6"), keep order:false, stats:pseudo + └─Selection_11(Probe) 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "测试%", 92) + └─TableRowIDScan_10 250.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a like '测试%' order by a,_tidb_rowid; +a b +测试 222 +测试 543 +测试Abc 324 +explain select * from t use index (ia) where a like '测_' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 250.00 root + ├─IndexRangeScan_9(Build) 250.00 cop[tikv] table:t, index:ia(a, b) range:["mK","mL"), keep order:false, stats:pseudo + └─Selection_11(Probe) 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "测_", 92) + └─TableRowIDScan_10 250.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a like '测_' order by a,_tidb_rowid; +a b +测试 222 +explain select * from t use index (ia) where a like '测测试 %' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 250.00 root + ├─IndexRangeScan_9(Build) 250.00 cop[tikv] table:t, index:ia(a, b) range:["mKmK\x8b\xd5","mKmK\x8b\xd6"), keep order:false, stats:pseudo + └─Selection_11(Probe) 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "测测试 %", 92) + └─TableRowIDScan_10 250.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a like '测测试 %' order by a,_tidb_rowid; +a b +测测试 2468 +测测试 99999 +explain select * from t use index (ia) where a like '测试a__' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 250.00 root + ├─IndexRangeScan_9(Build) 250.00 cop[tikv] table:t, index:ia(a, b) range:["mK\x8b\xd5\x00A","mK\x8b\xd5\x00B"), keep order:false, stats:pseudo + └─Selection_11(Probe) 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "测试a__", 92) + └─TableRowIDScan_10 250.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a like '测试a__' order by a,_tidb_rowid; +a b +测试Abc 324 +explain select * from t use index (ia) where a like '测试 __' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 250.00 root + ├─IndexRangeScan_9(Build) 250.00 cop[tikv] table:t, index:ia(a, b) range:["mK\x8b\xd5","mK\x8b\xd5\x00!"), keep order:false, stats:pseudo + └─Selection_11(Probe) 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, "测试 __", 92) + └─TableRowIDScan_10 250.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a like '测试 __' order by a,_tidb_rowid; +a b +测试 543 +explain select * from t use index (ia) where a like ' _' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 250.00 root + ├─IndexRangeScan_9(Build) 250.00 cop[tikv] table:t, index:ia(a, b) range:["","\x00!"), keep order:false, stats:pseudo + └─Selection_11(Probe) 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, " _", 92) + └─TableRowIDScan_10 250.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a like ' _' order by a,_tidb_rowid; +a b +explain select * from t use index (ia) where a like ' %' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 250.00 root + ├─IndexRangeScan_9(Build) 250.00 cop[tikv] table:t, index:ia(a, b) range:["","\x00 \x00 \x00!"), keep order:false, stats:pseudo + └─Selection_11(Probe) 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, " %", 92) + └─TableRowIDScan_10 250.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a like ' %' order by a,_tidb_rowid; +a b + 66666 +explain select * from t use index (ia) where a like ' 语言%%' order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 250.00 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 250.00 root + ├─IndexRangeScan_9(Build) 250.00 cop[tikv] table:t, index:ia(a, b) range:["\x00 \x8b\xed\x8a\x00","\x00 \x8b\xed\x8a\x01"), keep order:false, stats:pseudo + └─Selection_11(Probe) 250.00 cop[tikv] like(planner__core__range_scan_for_like.t.a, " 语言%%", 92) + └─TableRowIDScan_10 250.00 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a like ' 语言%%' order by a,_tidb_rowid; +a b + 语言 55555 +explain select * from t use index (ia) where a not in ('aabc','dd') order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 3583.33 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 3583.33 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_12 3583.33 root + ├─IndexRangeScan_9(Build) 3583.33 cop[tikv] table:t, index:ia(a, b) range:(NULL,"\x00D\x00D"), ("\x00D\x00D",+inf], keep order:false, stats:pseudo + └─Selection_11(Probe) 3583.33 cop[tikv] not(in(planner__core__range_scan_for_like.t.a, "aabc", "dd")) + └─TableRowIDScan_10 3583.33 cop[tikv] table:t keep order:false, stats:pseudo +select * from t use index (ia) where a not in ('aabc','dd') order by a,_tidb_rowid; +a b + 234 + 11111 + 66666 + 语 言 3579 + 语言 55555 +A 456 +Aa 456 +aab 456 +aabB 456 +aABBccdd 890 +你好 111 +测测试 2468 +测测试 99999 +测试 222 +测试 543 +测试Abc 324 +explain select * from t where a >= 'aabb' and a <= 'aabd' and b = 456 order by a,_tidb_rowid; +id estRows task access object operator info +Projection_6 0.01 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t.b +└─Sort_7 0.01 root planner__core__range_scan_for_like.t.a, planner__core__range_scan_for_like.t._tidb_rowid + └─IndexLookUp_16 0.01 root + ├─Selection_14(Build) 0.01 cop[tikv] eq(planner__core__range_scan_for_like.t.b, 456) + │ └─IndexRangeScan_12 10.00 cop[tikv] table:t, index:ia(a, b) range:["\x00A\x00A\x00B","\x00A\x00A\x00B"], keep order:false, stats:pseudo + └─Selection_15(Probe) 0.01 cop[tikv] ge(planner__core__range_scan_for_like.t.a, "aabb"), le(planner__core__range_scan_for_like.t.a, "aabd") + └─TableRowIDScan_13 0.01 cop[tikv] table:t keep order:false, stats:pseudo +select * from t where a >= 'aabb' and a <= 'aabd' and b = 456 order by a,_tidb_rowid; +a b +aabB 456 diff --git a/tests/integrationtest/t/planner/core/range_scan_for_like.test b/tests/integrationtest/t/planner/core/range_scan_for_like.test new file mode 100644 index 0000000000000..ff60df3f17584 --- /dev/null +++ b/tests/integrationtest/t/planner/core/range_scan_for_like.test @@ -0,0 +1,231 @@ +# Suite 1: utf8mb4_general_ci + normal index +create table t(a varchar(20) collate utf8mb4_general_ci, index ia(a)); +insert into t value('测试'),('测试Abc'),('测试 '),('你好'),('aABBccdd'),('Aa'),(''),(' '),(' '),(' 语言'),(' 语 言 '),('测测试 '),('测测试 '),(NULL); +# test cases for the pattern string cover: +# with/without wildcard +# start/end with wildcard +# [non-]ascii characters +# [only] contain empty string/space +explain select *, length(a) from t use index (ia) where a like '测试%' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like '测试%' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like '测%%' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like '测%%' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like '测%%试' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like '测%%试' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like '测试%%' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like '测试%%' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like '测试_' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like '测试_' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like '你好%' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like '你好%' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like 'aa' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like 'aa' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like 'aa%' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like 'aa%' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like 'aa%cc' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like 'aa%cc' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like '' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like '' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like ' ' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like ' ' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like 'aa%dd' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like 'aa%dd' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like 'aa%%dd' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like 'aa%%dd' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like 'aa_bccdd' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like 'aa_bccdd' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like '%%' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like '%%' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like ' %%' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like ' %%' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like ' %%语言' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like ' %%语言' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like ' 语 %' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like ' 语 %' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like ' 语 _' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like ' 语 _' order by a,_tidb_rowid; +drop table t; +# Suite 2: utf8mb4_unicode_ci + unique index +create table t(a varchar(20) collate utf8mb4_unicode_ci, unique index ia(a)); +insert into t value(''),('测试'),('测试abc'),('你好'),('aabbccdd'),(' 语言'),(' 语 言 '),('测测试 '); +# test cases for the pattern string are the same with Suite 1 +explain select *, length(a) from t use index (ia) where a like '测试%' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like '测试%' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like '测%%' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like '测%%' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like '测%%试' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like '测%%试' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like '测试%%' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like '测试%%' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like '测试_' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like '测试_' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like '你好%' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like '你好%' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like 'aa' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like 'aa' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like 'aa%' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like 'aa%' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like 'aa%cc' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like 'aa%cc' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like '' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like '' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like 'aa%dd' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like 'aa%dd' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like 'aa%%dd' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like 'aa%%dd' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like 'aa_bccdd' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like 'aa_bccdd' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like '%%' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like '%%' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like ' %%' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like ' %%' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like ' %%语言' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like ' %%语言' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like ' 语 %' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like ' 语 %' order by a,_tidb_rowid; +explain select *, length(a) from t use index (ia) where a like ' 语 _' order by a,_tidb_rowid; +select *, length(a) from t use index (ia) where a like ' 语 _' order by a,_tidb_rowid; +drop table t; +# Suite 3: utf8mb4_0900_ai_ci + ascii_bin + multi-column index + prefix index + primary key (clustered) +create table t(a varchar(20) collate utf8mb4_0900_ai_ci, b varchar(20) collate ascii_bin, c bigint, primary key(a(1), b) clustered); +insert into t (a, b, c) values +('测试1', 'asdfgh', 345346), +('你好2', 'qqwweerrrr', 987765), +('こんにちは3', 'zxcvbnn', 1111111), +('안녕하세요4', 'asdfgh ', 3333333333), +('Ciao5', ' asdfgh', 444400), +('Hola6', ' asdfgh ', 6666), +('Bonjour ', '', 888888888), +('Olá8', ' ', 9999999), +('Привет9', ' ', 321321), +('Hallo10', '12345', 35678); +explain select * from t use index (primary) where a like '测试%' and b like 'asd%' order by a,b; +select * from t use index (primary) where a like '测试%' and b like 'asd%' order by a,b; +explain select * from t use index (primary) where a like '测试1' and b like 'asdfgh %' order by a,b; +select * from t use index (primary) where a like '测试1' and b like 'asdfgh %' order by a,b; +explain select * from t use index (primary) where a like 'こんにち_' and b like 'zxc%' order by a,b; +select * from t use index (primary) where a like 'こんにち_' and b like 'zxc%' order by a,b; +explain select * from t use index (primary) where a like '안녕하세요%' and b like 'asd%' order by a,b; +select * from t use index (primary) where a like '안녕하세요%' and b like 'asd%' order by a,b; +explain select * from t use index (primary) where a like 'Ciáo%' and b like ' _%' order by a,b; +select * from t use index (primary) where a like 'Ciáo%' and b like ' _%' order by a,b; +explain select * from t use index (primary) where a like '%HoLa%' and b like ' asdfgh' order by a,b; +select * from t use index (primary) where a like '%HoLa%' and b like ' asdfgh' order by a,b; +explain select * from t use index (primary) where a like 'bonjour _%' and b like '' order by a,b; +select * from t use index (primary) where a like 'bonjour _%' and b like '' order by a,b; +explain select * from t use index (primary) where a like 'OLa%' and b like '_' order by a,b; +select * from t use index (primary) where a like 'OLa%' and b like '_' order by a,b; +explain select * from t use index (primary) where a like 'Приве__' and b like ' %' order by a,b; +select * from t use index (primary) where a like 'Приве__' and b like ' %' order by a,b; +explain select * from t use index (primary) where a like 'Hallo%' and b like '123%' order by a,b; +select * from t use index (primary) where a like 'Hallo%' and b like '123%' order by a,b; +drop table t; +# Suite 4: gbk_chinese_ci + latin1_bin + multi-column index + prefix index + primary key (nonclustered) +create table t(a varchar(20) collate gbk_chinese_ci, b varchar(20) collate latin1_bin, c bigint, primary key(a, b(5)) nonclustered); +insert into t (a, b, c) values +('测试1', 'asdfgh', 345346), +('你好2', 'qqwweerrrr', 987765), +('zxcvbnn',0xE38193E38293E381ABE381A1E381AF33, 1111111), +('asdfgh ', 0xEC9588EB8595ED9598EC84B8EC9A9434, 3333333333), +('Ciao5', ' asdfgh', 444400), +(' asdfgh ', 'Hola6', 6666), +('Bonjour ', '', 888888888), +('Olá8', ' ', 9999999), +('Привет9', ' ', 321321), +(' ', '12345', 35678); +set names utf8mb4; +explain select * from t use index (primary) where a like '测试%' and b like 'asd%' order by a,b; +select * from t use index (primary) where a like '测试%' and b like 'asd%' order by a,b; +explain select * from t use index (primary) where a like '测试1' and b like 'asdfgh %' order by a,b; +select * from t use index (primary) where a like '测试1' and b like 'asdfgh %' order by a,b; +set names latin1; +explain select * from t use index (primary) where b like 'こんにち_' and a like 'zxc%' order by a,b; +select * from t use index (primary) where b like 'こんにち_' and a like 'zxc%' order by a,b; +explain select * from t use index (primary) where b like '안녕하세요%' and a like 'asd%' order by a,b; +select * from t use index (primary) where b like '안녕하세요%' and a like 'asd%' order by a,b; +set names utf8mb4; +explain select * from t use index (primary) where a like 'Ciao%' and b like ' _%' order by a,b; +select * from t use index (primary) where a like 'Ciao%' and b like ' _%' order by a,b; +explain select * from t use index (primary) where b like 'HoLa%' and a like ' asdfgh' order by a,b; +select * from t use index (primary) where b like 'HoLa%' and a like ' asdfgh' order by a,b; +explain select * from t use index (primary) where a like 'bonjour _%' and b like '' order by a,b; +select * from t use index (primary) where a like 'bonjour _%' and b like '' order by a,b; +explain select * from t use index (primary) where a like 'OLá' and b like '_' order by a,b; +select * from t use index (primary) where a like 'OLá' and b like '_' order by a,b; +explain select * from t use index (primary) where a like 'Приве__' and b like ' %' order by a,b; +select * from t use index (primary) where a like 'Приве__' and b like ' %' order by a,b; +explain select * from t use index (primary) where a like ' %' and b like '123%' order by a,b; +select * from t use index (primary) where a like ' %' and b like '123%' order by a,b; +drop table t; +# Suite 5: utf8mb4_general_ci + prefix index +create table t(a varchar(20) collate utf8mb4_general_ci, b bigint, index ia(a(3),b)); +insert into t value +('测试',222), +('测试Abc',324), +('测试 ',543), +('你好',111), +('aABBccdd',890), +('A',456), +('Aa',456), +('aab',456), +('aabB',456), +('',234), +(' ',11111), +(' ',66666), +(' 语言',55555), +(' 语 言',3579), +('测测试 ',2468), +('测测试 ',99999), +(NULL,10); +explain select * from t use index (ia) where a > 'aabb' order by a,_tidb_rowid; +select * from t use index (ia) where a > 'aabb' order by a,_tidb_rowid; +explain select * from t use index (ia) where a > 'aab' order by a,_tidb_rowid; +select * from t use index (ia) where a > 'aab' order by a,_tidb_rowid; +explain select * from t use index (ia) where a > 'aa' order by a,_tidb_rowid; +select * from t use index (ia) where a > 'aa' order by a,_tidb_rowid; +explain select * from t use index (ia) where a < 'aabb' order by a,_tidb_rowid; +select * from t use index (ia) where a < 'aabb' order by a,_tidb_rowid; +explain select * from t use index (ia) where a < 'aab' order by a,_tidb_rowid; +select * from t use index (ia) where a < 'aab' order by a,_tidb_rowid; +explain select * from t use index (ia) where a < 'aa' order by a,_tidb_rowid; +select * from t use index (ia) where a < 'aa' order by a,_tidb_rowid; +explain select * from t use index (ia) where a != 'aa' order by a,_tidb_rowid; +select * from t use index (ia) where a != 'aa' order by a,_tidb_rowid; +explain select * from t use index (ia) where a != 'aaBbc' order by a,_tidb_rowid; +select * from t use index (ia) where a != 'aaBbc' order by a,_tidb_rowid; +explain select * from t use index (ia) where a like '测试abc' order by a,_tidb_rowid; +select * from t use index (ia) where a like '测试abc' order by a,_tidb_rowid; +explain select * from t use index (ia) where a = '测试abc' order by a,_tidb_rowid; +select * from t use index (ia) where a = '测试abc' order by a,_tidb_rowid; +explain select * from t use index (ia) where a like 'aa' order by a,_tidb_rowid; +select * from t use index (ia) where a like 'aa' order by a,_tidb_rowid; +explain select * from t use index (ia) where a = 'aa' order by a,_tidb_rowid; +select * from t use index (ia) where a = 'aa' order by a,_tidb_rowid; +explain select * from t use index (ia) where a like '测测试 ' order by a,_tidb_rowid; +select * from t use index (ia) where a like '测测试 ' order by a,_tidb_rowid; +explain select * from t use index (ia) where a = '测测试 ' order by a,_tidb_rowid; +select * from t use index (ia) where a = '测测试 ' order by a,_tidb_rowid; +explain select * from t use index (ia) where a like ' 语 言' order by a,_tidb_rowid; +select * from t use index (ia) where a like ' 语 言' order by a,_tidb_rowid; +explain select * from t use index (ia) where a = ' 语 言' order by a,_tidb_rowid; +select * from t use index (ia) where a = ' 语 言' order by a,_tidb_rowid; +explain select * from t use index (ia) where a like '测试%' order by a,_tidb_rowid; +select * from t use index (ia) where a like '测试%' order by a,_tidb_rowid; +explain select * from t use index (ia) where a like '测_' order by a,_tidb_rowid; +select * from t use index (ia) where a like '测_' order by a,_tidb_rowid; +explain select * from t use index (ia) where a like '测测试 %' order by a,_tidb_rowid; +select * from t use index (ia) where a like '测测试 %' order by a,_tidb_rowid; +explain select * from t use index (ia) where a like '测试a__' order by a,_tidb_rowid; +select * from t use index (ia) where a like '测试a__' order by a,_tidb_rowid; +explain select * from t use index (ia) where a like '测试 __' order by a,_tidb_rowid; +select * from t use index (ia) where a like '测试 __' order by a,_tidb_rowid; +explain select * from t use index (ia) where a like ' _' order by a,_tidb_rowid; +select * from t use index (ia) where a like ' _' order by a,_tidb_rowid; +explain select * from t use index (ia) where a like ' %' order by a,_tidb_rowid; +select * from t use index (ia) where a like ' %' order by a,_tidb_rowid; +explain select * from t use index (ia) where a like ' 语言%%' order by a,_tidb_rowid; +select * from t use index (ia) where a like ' 语言%%' order by a,_tidb_rowid; +explain select * from t use index (ia) where a not in ('aabc','dd') order by a,_tidb_rowid; +select * from t use index (ia) where a not in ('aabc','dd') order by a,_tidb_rowid; +explain select * from t where a >= 'aabb' and a <= 'aabd' and b = 456 order by a,_tidb_rowid; +select * from t where a >= 'aabb' and a <= 'aabd' and b = 456 order by a,_tidb_rowid; diff --git a/util/ranger/checker.go b/util/ranger/checker.go index 0468d3c92472e..c33e9e2221d28 100644 --- a/util/ranger/checker.go +++ b/util/ranger/checker.go @@ -139,16 +139,6 @@ func (c *conditionChecker) checkScalarFunction(scalar *expression.ScalarFunction func (c *conditionChecker) checkLikeFunc(scalar *expression.ScalarFunction) (isAccessCond, shouldReserve bool) { _, collation := scalar.CharsetAndCollation() - if collate.NewCollationEnabled() && !collate.IsBinCollation(collation) { - // The algorithm constructs the range in byte-level: for example, ab% is mapped to [ab, ac] by adding 1 to the last byte. - // However, this is incorrect for non-binary collation strings because the sort key order is not the same as byte order. - // For example, "`%" is mapped to the range [`, a](where ` is 0x60 and a is 0x61). - // Because the collation utf8_general_ci is case-insensitive, a and A have the same sort key. - // Finally, the range comes to be [`, A], which is actually an empty range. - // See https://github.com/pingcap/tidb/issues/31174 for more details. - // In short, when the column type is non-binary collation string, we cannot use `like` expressions to generate the range. - return false, true - } if !collate.CompatibleCollate(scalar.GetArgs()[0].GetType().GetCollate(), collation) { return false, true } diff --git a/util/ranger/detacher.go b/util/ranger/detacher.go index 8ac79b9502ba8..7a5bfc915bec3 100644 --- a/util/ranger/detacher.go +++ b/util/ranger/detacher.go @@ -242,7 +242,7 @@ func compareCNFItemRangeResult(curResult, bestResult *cnfItemRangeResult) (curIs // e.g, for input CNF expressions ((a,b) in ((1,1),(2,2))) and a > 1 and ((a,b,c) in (1,1,1),(2,2,2)) // ((a,b,c) in (1,1,1),(2,2,2)) would be extracted. func extractBestCNFItemRanges(sctx sessionctx.Context, conds []expression.Expression, cols []*expression.Column, - lengths []int, rangeMaxSize int64) (*cnfItemRangeResult, []*valueInfo, error) { + lengths []int, rangeMaxSize int64, convertToSortKey bool) (*cnfItemRangeResult, []*valueInfo, error) { if len(conds) < 2 { return nil, nil, nil } @@ -261,7 +261,7 @@ func extractBestCNFItemRanges(sctx sessionctx.Context, conds []expression.Expres // We build ranges for `(a,b) in ((1,1),(1,2))` and get `[1 1, 1 1] [1 2, 1 2]`, which are point ranges and we can // append `c = 1` to the point ranges. However, if we choose to merge consecutive ranges here, we get `[1 1, 1 2]`, // which are not point ranges, and we cannot append `c = 1` anymore. - res, err := detachCondAndBuildRangeWithoutMerging(sctx, tmpConds, cols, lengths, rangeMaxSize) + res, err := detachCondAndBuildRangeWithoutMerging(sctx, tmpConds, cols, lengths, rangeMaxSize, convertToSortKey) if err != nil { return nil, nil, err } @@ -376,7 +376,7 @@ func (d *rangeDetacher) detachCNFCondAndBuildRangeForIndex(conditions []expressi optPrefixIndexSingleScan: d.sctx.GetSessionVars().OptPrefixIndexSingleScan, } if considerDNF { - bestCNFItemRes, columnValues, err := extractBestCNFItemRanges(d.sctx, conditions, d.cols, d.lengths, d.rangeMaxSize) + bestCNFItemRes, columnValues, err := extractBestCNFItemRanges(d.sctx, conditions, d.cols, d.lengths, d.rangeMaxSize, d.convertToSortKey) if err != nil { return nil, err } @@ -631,12 +631,22 @@ func ExtractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Ex } // Multiple Eq/In conditions for one column in CNF, apply intersection on them // Lazily compute the points for the previously visited Eq/In + newTp := newFieldType(cols[offset].GetType()) collator := collate.GetCollator(cols[offset].GetType().GetCollate()) if mergedAccesses[offset] == nil { mergedAccesses[offset] = accesses[offset] +<<<<<<< HEAD:util/ranger/detacher.go points[offset] = rb.build(accesses[offset], collator) } points[offset] = rb.intersection(points[offset], rb.build(cond, collator), collator) +======= + // Note that this is a relatively special usage of build(). We will restore the points back to Expression for + // later use and may build the Expression to points again. + // We need to keep the original value here, which means we neither cut prefix nor convert to sort key. + points[offset] = rb.build(accesses[offset], newTp, types.UnspecifiedLength, false) + } + points[offset] = rb.intersection(points[offset], rb.build(cond, newTp, types.UnspecifiedLength, false), collator) +>>>>>>> e053c27f068 (util/ranger: support use `like` to build range for new collation columns (#48522)):pkg/util/ranger/detacher.go if len(points[offset]) == 0 { // Early termination if false expression found if expression.MaybeOverOptimized4PlanCache(sctx, conditions) { // `a>@x and a<@y` --> `invalid-range if @x>=@y` @@ -776,9 +786,14 @@ func (d *rangeDetacher) detachDNFCondAndBuildRangeForIndex(condition *expression if shouldReserve { hasResidual = true } +<<<<<<< HEAD:util/ranger/detacher.go points := rb.build(item, collate.GetCollator(newTpSlice[0].GetCollate())) +======= + points := rb.build(item, newTpSlice[0], d.lengths[0], d.convertToSortKey) + tmpNewTp := convertStringFTToBinaryCollate(newTpSlice[0]) +>>>>>>> e053c27f068 (util/ranger: support use `like` to build range for new collation columns (#48522)):pkg/util/ranger/detacher.go // TODO: restrict the mem usage of ranges - ranges, rangeFallback, err := points2Ranges(d.sctx, points, newTpSlice[0], d.rangeMaxSize) + ranges, rangeFallback, err := points2Ranges(d.sctx, points, tmpNewTp, d.rangeMaxSize) if err != nil { return nil, nil, nil, false, errors.Trace(err) } @@ -874,6 +889,7 @@ func DetachCondAndBuildRangeForIndex(sctx sessionctx.Context, conditions []expre cols: cols, lengths: lengths, mergeConsecutive: true, + convertToSortKey: true, rangeMaxSize: rangeMaxSize, } return d.detachCondAndBuildRangeForCols() @@ -882,13 +898,14 @@ func DetachCondAndBuildRangeForIndex(sctx sessionctx.Context, conditions []expre // detachCondAndBuildRangeWithoutMerging detaches the index filters from table filters and uses them to build ranges. // When building ranges, it doesn't merge consecutive ranges. func detachCondAndBuildRangeWithoutMerging(sctx sessionctx.Context, conditions []expression.Expression, cols []*expression.Column, - lengths []int, rangeMaxSize int64) (*DetachRangeResult, error) { + lengths []int, rangeMaxSize int64, convertToSortKey bool) (*DetachRangeResult, error) { d := &rangeDetacher{ sctx: sctx, allConds: conditions, cols: cols, lengths: lengths, mergeConsecutive: false, + convertToSortKey: convertToSortKey, rangeMaxSize: rangeMaxSize, } return d.detachCondAndBuildRangeForCols() @@ -900,7 +917,7 @@ func detachCondAndBuildRangeWithoutMerging(sctx sessionctx.Context, conditions [ // The returned values are encapsulated into a struct DetachRangeResult, see its comments for explanation. func DetachCondAndBuildRangeForPartition(sctx sessionctx.Context, conditions []expression.Expression, cols []*expression.Column, lengths []int, rangeMaxSize int64) (*DetachRangeResult, error) { - return detachCondAndBuildRangeWithoutMerging(sctx, conditions, cols, lengths, rangeMaxSize) + return detachCondAndBuildRangeWithoutMerging(sctx, conditions, cols, lengths, rangeMaxSize, false) } type rangeDetacher struct { @@ -909,6 +926,7 @@ type rangeDetacher struct { cols []*expression.Column lengths []int mergeConsecutive bool + convertToSortKey bool rangeMaxSize int64 } @@ -955,6 +973,7 @@ func DetachSimpleCondAndBuildRangeForIndex(sctx sessionctx.Context, conditions [ cols: cols, lengths: lengths, mergeConsecutive: true, + convertToSortKey: true, rangeMaxSize: rangeMaxSize, } res, err := d.detachCNFCondAndBuildRangeForIndex(conditions, newTpSlice, false) diff --git a/util/ranger/points.go b/util/ranger/points.go index 6bf08c3c0f510..dc387aea58bdb 100644 --- a/util/ranger/points.go +++ b/util/ranger/points.go @@ -20,6 +20,7 @@ import ( "sort" "github.com/pingcap/errors" +<<<<<<< HEAD:util/ranger/points.go "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/parser/ast" @@ -30,6 +31,19 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/dbterror" +======= + "github.com/pingcap/tidb/pkg/errno" + "github.com/pingcap/tidb/pkg/expression" + "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/parser/charset" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tidb/pkg/util/chunk" + "github.com/pingcap/tidb/pkg/util/collate" + "github.com/pingcap/tidb/pkg/util/dbterror" + "github.com/pingcap/tidb/pkg/util/hack" +>>>>>>> e053c27f068 (util/ranger: support use `like` to build range for new collation columns (#48522)):pkg/util/ranger/points.go ) // Error instances. @@ -134,6 +148,50 @@ func rangePointEqualValueLess(a, b *point) bool { return a.excl && !b.excl } +func pointsConvertToSortKey(sctx sessionctx.Context, inputPs []*point, newTp *types.FieldType) ([]*point, error) { + // Only handle normal string type here. + // Currently, set won't be pushed down and it shouldn't reach here in theory. + // For enum, we have separate logic for it, like handleEnumFromBinOp(). For now, it only supports point range, + // intervals are not supported. So we also don't need to handle enum here. + if newTp.EvalType() != types.ETString || + newTp.GetType() == mysql.TypeEnum || + newTp.GetType() == mysql.TypeSet { + return inputPs, nil + } + ps := make([]*point, 0, len(inputPs)) + for _, p := range inputPs { + np, err := pointConvertToSortKey(sctx, p, newTp, true) + if err != nil { + return nil, err + } + ps = append(ps, np) + } + return ps, nil +} + +func pointConvertToSortKey( + sctx sessionctx.Context, + inputP *point, + newTp *types.FieldType, + trimTrailingSpace bool, +) (*point, error) { + p, err := convertPoint(sctx, inputP, newTp) + if err != nil { + return nil, err + } + if p.value.Kind() != types.KindString || newTp.GetCollate() == charset.CollationBin || !collate.NewCollationEnabled() { + return p, nil + } + sortKey := p.value.GetBytes() + if !trimTrailingSpace { + sortKey = collate.GetCollator(newTp.GetCollate()).KeyWithoutTrimRightSpace(string(hack.String(sortKey))) + } else { + sortKey = collate.GetCollator(newTp.GetCollate()).Key(string(hack.String(sortKey))) + } + + return &point{value: types.NewBytesDatum(sortKey), excl: p.excl, start: p.start}, nil +} + func (r *pointSorter) Swap(i, j int) { r.points[i], r.points[j] = r.points[j], r.points[i] } @@ -186,12 +244,32 @@ type builder struct { sc *stmtctx.StatementContext } +<<<<<<< HEAD:util/ranger/points.go func (r *builder) build(expr expression.Expression, collator collate.Collator) []*point { +======= +// build converts Expression on one column into point, which can be further built into Range. +// If the input prefixLen is not types.UnspecifiedLength, it means it's for a prefix column in a prefix index. In such +// cases, we should cut the prefix and adjust the exclusiveness. Ref: cutPrefixForPoints(). +// convertToSortKey indicates whether the string values should be converted to sort key. +// Converting to sort key can make `like` function be built into Range for new collation column. But we can't restore +// the original value from the sort key, so the usage of the result may be limited, like when you need to restore the +// result points back to Expression. +func (r *builder) build( + expr expression.Expression, + newTp *types.FieldType, + prefixLen int, + convertToSortKey bool, +) []*point { +>>>>>>> e053c27f068 (util/ranger: support use `like` to build range for new collation columns (#48522)):pkg/util/ranger/points.go switch x := expr.(type) { case *expression.Column: return r.buildFromColumn() case *expression.ScalarFunction: +<<<<<<< HEAD:util/ranger/points.go return r.buildFromScalarFunc(x, collator) +======= + return r.buildFromScalarFunc(x, newTp, prefixLen, convertToSortKey) +>>>>>>> e053c27f068 (util/ranger: support use `like` to build range for new collation columns (#48522)):pkg/util/ranger/points.go case *expression.Constant: return r.buildFromConstant(x) } @@ -232,7 +310,16 @@ func (*builder) buildFromColumn() []*point { return []*point{startPoint1, endPoint1, startPoint2, endPoint2} } +<<<<<<< HEAD:util/ranger/points.go func (r *builder) buildFromBinOp(expr *expression.ScalarFunction) []*point { +======= +func (r *builder) buildFromBinOp( + expr *expression.ScalarFunction, + newTp *types.FieldType, + prefixLen int, + convertToSortKey bool, +) []*point { +>>>>>>> e053c27f068 (util/ranger: support use `like` to build range for new collation columns (#48522)):pkg/util/ranger/points.go // This has been checked that the binary operation is comparison operation, and one of // the operand is column name expression. var ( @@ -371,7 +458,19 @@ func (r *builder) buildFromBinOp(expr *expression.ScalarFunction) []*point { endPoint := &point{value: types.MaxValueDatum()} return []*point{startPoint, endPoint} } +<<<<<<< HEAD:util/ranger/points.go return nil +======= + cutPrefixForPoints(res, prefixLen, ft) + if convertToSortKey { + res, err = pointsConvertToSortKey(r.sctx, res, newTp) + if err != nil { + r.err = err + return getFullRange() + } + } + return res +>>>>>>> e053c27f068 (util/ranger: support use `like` to build range for new collation columns (#48522)):pkg/util/ranger/points.go } // handleUnsignedCol handles the case when unsigned column meets negative value. @@ -553,7 +652,16 @@ func (*builder) buildFromIsFalse(_ *expression.ScalarFunction, isNot int) []*poi return []*point{startPoint, endPoint} } +<<<<<<< HEAD:util/ranger/points.go func (r *builder) buildFromIn(expr *expression.ScalarFunction) ([]*point, bool) { +======= +func (r *builder) buildFromIn( + expr *expression.ScalarFunction, + newTp *types.FieldType, + prefixLen int, + convertToSortKey bool, +) ([]*point, bool) { +>>>>>>> e053c27f068 (util/ranger: support use `like` to build range for new collation columns (#48522)):pkg/util/ranger/points.go list := expr.GetArgs()[1:] rangePoints := make([]*point, 0, len(list)*2) hasNull := false @@ -629,10 +737,32 @@ func (r *builder) buildFromIn(expr *expression.ScalarFunction) ([]*point, bool) if curPos > 0 { curPos++ } +<<<<<<< HEAD:util/ranger/points.go return rangePoints[:curPos], hasNull } func (r *builder) newBuildFromPatternLike(expr *expression.ScalarFunction) []*point { +======= + rangePoints = rangePoints[:curPos] + cutPrefixForPoints(rangePoints, prefixLen, ft) + var err error + if convertToSortKey { + rangePoints, err = pointsConvertToSortKey(r.sctx, rangePoints, newTp) + if err != nil { + r.err = err + return getFullRange(), false + } + } + return rangePoints, hasNull +} + +func (r *builder) newBuildFromPatternLike( + expr *expression.ScalarFunction, + newTp *types.FieldType, + prefixLen int, + convertToSortKey bool, +) []*point { +>>>>>>> e053c27f068 (util/ranger: support use `like` to build range for new collation columns (#48522)):pkg/util/ranger/points.go _, collation := expr.CharsetAndCollation() if !collate.CompatibleCollate(expr.GetArgs()[0].GetType().GetCollate(), collation) { return getFullRange() @@ -648,10 +778,23 @@ func (r *builder) newBuildFromPatternLike(expr *expression.ScalarFunction) []*po r.err = errors.Trace(err) return getFullRange() } + // non-exceptional return case 1: empty pattern if pattern == "" { startPoint := &point{value: types.NewStringDatum(""), start: true} endPoint := &point{value: types.NewStringDatum("")} +<<<<<<< HEAD:util/ranger/points.go return []*point{startPoint, endPoint} +======= + res := []*point{startPoint, endPoint} + if convertToSortKey { + res, err = pointsConvertToSortKey(r.sctx, res, newTp) + if err != nil { + r.err = err + return getFullRange() + } + } + return res +>>>>>>> e053c27f068 (util/ranger: support use `like` to build range for new collation columns (#48522)):pkg/util/ranger/points.go } lowValue := make([]byte, 0, len(pattern)) edt, err := expr.GetArgs()[2].(*expression.Constant).Eval(chunk.Row{}) @@ -692,11 +835,14 @@ func (r *builder) newBuildFromPatternLike(expr *expression.ScalarFunction) []*po } lowValue = append(lowValue, pattern[i]) } + // non-exceptional return case 2: no characters before the wildcard if len(lowValue) == 0 { return []*point{{value: types.MinNotNullDatum(), start: true}, {value: types.MaxValueDatum()}} } + // non-exceptional return case 3: pattern contains valid characters and doesn't contain the wildcard if isExactMatch { val := types.NewCollationStringDatum(string(lowValue), tpOfPattern.GetCollate()) +<<<<<<< HEAD:util/ranger/points.go return []*point{{value: val, start: true}, {value: val}} } startPoint := &point{start: true, excl: exclude} @@ -705,15 +851,67 @@ func (r *builder) newBuildFromPatternLike(expr *expression.ScalarFunction) []*po copy(highValue, lowValue) endPoint := &point{excl: true} for i := len(highValue) - 1; i >= 0; i-- { +======= + startPoint := &point{value: val, start: true} + endPoint := &point{value: val} + res := []*point{startPoint, endPoint} + cutPrefixForPoints(res, prefixLen, tpOfPattern) + if convertToSortKey { + res, err = pointsConvertToSortKey(r.sctx, res, newTp) + if err != nil { + r.err = err + return getFullRange() + } + } + return res + } + + // non-exceptional return case 4: pattern contains valid characters and contains the wildcard + + // non-exceptional return case 4-1 + // If it's not a _bin or binary collation, and we don't convert the value to the sort key, we can't build + // a range for the wildcard. + if !convertToSortKey && + !collate.IsBinCollation(tpOfPattern.GetCollate()) { + return []*point{{value: types.MinNotNullDatum(), start: true}, {value: types.MaxValueDatum()}} + } + + // non-exceptional return case 4-2: build a range for the wildcard + // the end_key is sortKey(start_value) + 1 + originalStartPoint := &point{start: true, excl: exclude} + originalStartPoint.value.SetBytesAsString(lowValue, tpOfPattern.GetCollate(), uint32(tpOfPattern.GetFlen())) + cutPrefixForPoints([]*point{originalStartPoint}, prefixLen, tpOfPattern) + + // If we don't trim the trailing spaces, which means using KeyWithoutTrimRightSpace() instead of Key(), we can build + // a smaller range for better performance, e.g., LIKE ' %'. + // However, if it's a PAD SPACE collation, we must trim the trailing spaces for the start point to ensure the correctness. + // Because the trailing spaces are trimmed in the stored index key. For example, for LIKE 'abc %' on utf8mb4_bin + // column, the start key should be 'abd' instead of 'abc ', but the end key can be 'abc!'. ( ' ' is 32 and '!' is 33 + // in ASCII) + shouldTrimTrailingSpace := isPadSpaceCollation(collation) + startPoint, err := pointConvertToSortKey(r.sctx, originalStartPoint, newTp, shouldTrimTrailingSpace) + if err != nil { + r.err = errors.Trace(err) + return getFullRange() + } + sortKeyPointWithoutTrim, err := pointConvertToSortKey(r.sctx, originalStartPoint, newTp, false) + if err != nil { + r.err = errors.Trace(err) + return getFullRange() + } + sortKeyWithoutTrim := append([]byte{}, sortKeyPointWithoutTrim.value.GetBytes()...) + endPoint := &point{value: types.MaxValueDatum(), excl: true} + for i := len(sortKeyWithoutTrim) - 1; i >= 0; i-- { +>>>>>>> e053c27f068 (util/ranger: support use `like` to build range for new collation columns (#48522)):pkg/util/ranger/points.go // Make the end point value more than the start point value, // and the length of the end point value is the same as the length of the start point value. // e.g., the start point value is "abc", so the end point value is "abd". - highValue[i]++ - if highValue[i] != 0 { - endPoint.value.SetBytesAsString(highValue, tpOfPattern.GetCollate(), uint32(tpOfPattern.GetFlen())) + sortKeyWithoutTrim[i]++ + if sortKeyWithoutTrim[i] != 0 { + endPoint.value.SetBytes(sortKeyWithoutTrim) break } - // If highValue[i] is 255 and highValue[i]++ is 0, then the end point value is max value. + // If sortKeyWithoutTrim[i] is 255 and sortKeyWithoutTrim[i]++ is 0, then the end point value is max value. if i == 0 { endPoint.value = types.MaxValueDatum() } @@ -729,7 +927,16 @@ func isPadSpaceCollation(collation string) bool { return collation != charset.CollationBin } +<<<<<<< HEAD:util/ranger/points.go func (r *builder) buildFromNot(expr *expression.ScalarFunction) []*point { +======= +func (r *builder) buildFromNot( + expr *expression.ScalarFunction, + newTp *types.FieldType, + prefixLen int, + convertToSortKey bool, +) []*point { +>>>>>>> e053c27f068 (util/ranger: support use `like` to build range for new collation columns (#48522)):pkg/util/ranger/points.go switch n := expr.FuncName.L; n { case ast.IsTruthWithoutNull: return r.buildFromIsTrue(expr, 1, false) @@ -742,7 +949,18 @@ func (r *builder) buildFromNot(expr *expression.ScalarFunction) []*point { isUnsignedIntCol bool nonNegativePos int ) +<<<<<<< HEAD:util/ranger/points.go rangePoints, hasNull := r.buildFromIn(expr) +======= + // Note that we must handle the cutting prefix and converting to sort key in buildFromNot, because if we cut the + // prefix inside buildFromIn(), the inversion logic here would make an incomplete and wrong range. + // For example, for index col(1), col NOT IN ('aaa', 'bbb'), if we cut the prefix in buildFromIn(), we would get + // ['a', 'a'], ['b', 'b'] from there. Then after in this function we would get ['' 'a'), ('a', 'b'), ('b', +inf] + // as the result. This is wrong because data like 'ab' would be missed. Actually we are unable to build a range + // for this case. + // So we must cut the prefix in this function, therefore converting to sort key must also be done here. + rangePoints, hasNull := r.buildFromIn(expr, newTp, types.UnspecifiedLength, false) +>>>>>>> e053c27f068 (util/ranger: support use `like` to build range for new collation columns (#48522)):pkg/util/ranger/points.go if hasNull { return nil } @@ -768,6 +986,18 @@ func (r *builder) buildFromNot(expr *expression.ScalarFunction) []*point { // Append the interval (last element, max value]. retRangePoints = append(retRangePoints, &point{value: previousValue, start: true, excl: true}) retRangePoints = append(retRangePoints, &point{value: types.MaxValueDatum()}) +<<<<<<< HEAD:util/ranger/points.go +======= + cutPrefixForPoints(retRangePoints, prefixLen, expr.GetArgs()[0].GetType()) + if convertToSortKey { + var err error + retRangePoints, err = pointsConvertToSortKey(r.sctx, retRangePoints, newTp) + if err != nil { + r.err = err + return getFullRange() + } + } +>>>>>>> e053c27f068 (util/ranger: support use `like` to build range for new collation columns (#48522)):pkg/util/ranger/points.go return retRangePoints case ast.Like: // Pattern not like is not supported. @@ -784,6 +1014,7 @@ func (r *builder) buildFromNot(expr *expression.ScalarFunction) []*point { return getFullRange() } +<<<<<<< HEAD:util/ranger/points.go func (r *builder) buildFromScalarFunc(expr *expression.ScalarFunction, collator collate.Collator) []*point { switch op := expr.FuncName.L; op { case ast.GE, ast.GT, ast.LT, ast.LE, ast.EQ, ast.NE, ast.NullEQ: @@ -792,6 +1023,29 @@ func (r *builder) buildFromScalarFunc(expr *expression.ScalarFunction, collator return r.intersection(r.build(expr.GetArgs()[0], collator), r.build(expr.GetArgs()[1], collator), collator) case ast.LogicOr: return r.union(r.build(expr.GetArgs()[0], collator), r.build(expr.GetArgs()[1], collator), collator) +======= +func (r *builder) buildFromScalarFunc( + expr *expression.ScalarFunction, + newTp *types.FieldType, + prefixLen int, + convertToSortKey bool, +) []*point { + switch op := expr.FuncName.L; op { + case ast.GE, ast.GT, ast.LT, ast.LE, ast.EQ, ast.NE, ast.NullEQ: + return r.buildFromBinOp(expr, newTp, prefixLen, convertToSortKey) + case ast.LogicAnd: + collator := collate.GetCollator(newTp.GetCollate()) + if convertToSortKey { + collator = collate.GetCollator(charset.CollationBin) + } + return r.intersection(r.build(expr.GetArgs()[0], newTp, prefixLen, convertToSortKey), r.build(expr.GetArgs()[1], newTp, prefixLen, convertToSortKey), collator) + case ast.LogicOr: + collator := collate.GetCollator(newTp.GetCollate()) + if convertToSortKey { + collator = collate.GetCollator(charset.CollationBin) + } + return r.union(r.build(expr.GetArgs()[0], newTp, prefixLen, convertToSortKey), r.build(expr.GetArgs()[1], newTp, prefixLen, convertToSortKey), collator) +>>>>>>> e053c27f068 (util/ranger: support use `like` to build range for new collation columns (#48522)):pkg/util/ranger/points.go case ast.IsTruthWithoutNull: return r.buildFromIsTrue(expr, 0, false) case ast.IsTruthWithNull: @@ -799,25 +1053,42 @@ func (r *builder) buildFromScalarFunc(expr *expression.ScalarFunction, collator case ast.IsFalsity: return r.buildFromIsFalse(expr, 0) case ast.In: +<<<<<<< HEAD:util/ranger/points.go retPoints, _ := r.buildFromIn(expr) return retPoints case ast.Like: return r.newBuildFromPatternLike(expr) +======= + retPoints, _ := r.buildFromIn(expr, newTp, prefixLen, convertToSortKey) + return retPoints + case ast.Like: + return r.newBuildFromPatternLike(expr, newTp, prefixLen, convertToSortKey) +>>>>>>> e053c27f068 (util/ranger: support use `like` to build range for new collation columns (#48522)):pkg/util/ranger/points.go case ast.IsNull: startPoint := &point{start: true} endPoint := &point{} return []*point{startPoint, endPoint} case ast.UnaryNot: +<<<<<<< HEAD:util/ranger/points.go return r.buildFromNot(expr.GetArgs()[0].(*expression.ScalarFunction)) +======= + return r.buildFromNot(expr.GetArgs()[0].(*expression.ScalarFunction), newTp, prefixLen, convertToSortKey) +>>>>>>> e053c27f068 (util/ranger: support use `like` to build range for new collation columns (#48522)):pkg/util/ranger/points.go } return nil } +// We need an input collator because our (*Datum).Compare(), which is used in this method, needs an explicit collator +// input to handle comparison for string and bytes. +// Note that if the points are converted to sort key, the collator should be set to charset.CollationBin. func (r *builder) intersection(a, b []*point, collator collate.Collator) []*point { return r.merge(a, b, false, collator) } +// We need an input collator because our (*Datum).Compare(), which is used in this method, needs an explicit collator +// input to handle comparison for string and bytes. +// Note that if the points are converted to sort key, the collator should be set to charset.CollationBin. func (r *builder) union(a, b []*point, collator collate.Collator) []*point { return r.merge(a, b, true, collator) } diff --git a/util/ranger/ranger.go b/util/ranger/ranger.go index ad83f68ebd258..d0414e7750153 100644 --- a/util/ranger/ranger.go +++ b/util/ranger/ranger.go @@ -410,11 +410,20 @@ func points2TableRanges(sctx sessionctx.Context, rangePoints []*point, tp *types // The second return value is the conditions used to build ranges and the third return value is the remained conditions. func buildColumnRange(accessConditions []expression.Expression, sctx sessionctx.Context, tp *types.FieldType, tableRange bool, colLen int, rangeMaxSize int64) (Ranges, []expression.Expression, []expression.Expression, error) { +<<<<<<< HEAD:util/ranger/ranger.go rb := builder{sc: sctx.GetSessionVars().StmtCtx} rangePoints := getFullRange() for _, cond := range accessConditions { collator := collate.GetCollator(tp.GetCollate()) rangePoints = rb.intersection(rangePoints, rb.build(cond, collator), collator) +======= + rb := builder{sctx: sctx} + newTp := newFieldType(tp) + rangePoints := getFullRange() + for _, cond := range accessConditions { + collator := collate.GetCollator(charset.CollationBin) + rangePoints = rb.intersection(rangePoints, rb.build(cond, newTp, colLen, true), collator) +>>>>>>> e053c27f068 (util/ranger: support use `like` to build range for new collation columns (#48522)):pkg/util/ranger/ranger.go if rb.err != nil { return nil, nil, nil, errors.Trace(rb.err) } @@ -424,7 +433,7 @@ func buildColumnRange(accessConditions []expression.Expression, sctx sessionctx. rangeFallback bool err error ) - newTp := newFieldType(tp) + newTp = convertStringFTToBinaryCollate(newTp) if tableRange { ranges, rangeFallback, err = points2TableRanges(sctx, rangePoints, newTp, rangeMaxSize) } else { @@ -492,14 +501,19 @@ func (d *rangeDetacher) buildRangeOnColsByCNFCond(newTp []*types.FieldType, eqAn ) for i := 0; i < eqAndInCount; i++ { // Build ranges for equal or in access conditions. +<<<<<<< HEAD:util/ranger/ranger.go point := rb.build(accessConds[i], collate.GetCollator(newTp[i].GetCollate())) +======= + point := rb.build(accessConds[i], newTp[i], d.lengths[i], d.convertToSortKey) +>>>>>>> e053c27f068 (util/ranger: support use `like` to build range for new collation columns (#48522)):pkg/util/ranger/ranger.go if rb.err != nil { return nil, nil, nil, errors.Trace(rb.err) } + tmpNewTp := convertStringFTToBinaryCollate(newTp[i]) if i == 0 { - ranges, rangeFallback, err = points2Ranges(d.sctx, point, newTp[i], d.rangeMaxSize) + ranges, rangeFallback, err = points2Ranges(d.sctx, point, tmpNewTp, d.rangeMaxSize) } else { - ranges, rangeFallback, err = appendPoints2Ranges(d.sctx, ranges, point, newTp[i], d.rangeMaxSize) + ranges, rangeFallback, err = appendPoints2Ranges(d.sctx, ranges, point, tmpNewTp, d.rangeMaxSize) } if err != nil { return nil, nil, nil, errors.Trace(err) @@ -513,15 +527,30 @@ func (d *rangeDetacher) buildRangeOnColsByCNFCond(newTp []*types.FieldType, eqAn // Build rangePoints for non-equal access conditions. for i := eqAndInCount; i < len(accessConds); i++ { collator := collate.GetCollator(newTp[eqAndInCount].GetCollate()) +<<<<<<< HEAD:util/ranger/ranger.go rangePoints = rb.intersection(rangePoints, rb.build(accessConds[i], collator), collator) +======= + if d.convertToSortKey { + collator = collate.GetCollator(charset.CollationBin) + } + rangePoints = rb.intersection(rangePoints, rb.build(accessConds[i], newTp[eqAndInCount], d.lengths[eqAndInCount], d.convertToSortKey), collator) +>>>>>>> e053c27f068 (util/ranger: support use `like` to build range for new collation columns (#48522)):pkg/util/ranger/ranger.go if rb.err != nil { return nil, nil, nil, errors.Trace(rb.err) } } + var tmpNewTp *types.FieldType + if eqAndInCount == 0 || eqAndInCount < len(accessConds) { + if d.convertToSortKey { + tmpNewTp = convertStringFTToBinaryCollate(newTp[eqAndInCount]) + } else { + tmpNewTp = newTp[eqAndInCount] + } + } if eqAndInCount == 0 { - ranges, rangeFallback, err = points2Ranges(d.sctx, rangePoints, newTp[0], d.rangeMaxSize) + ranges, rangeFallback, err = points2Ranges(d.sctx, rangePoints, tmpNewTp, d.rangeMaxSize) } else if eqAndInCount < len(accessConds) { - ranges, rangeFallback, err = appendPoints2Ranges(d.sctx, ranges, rangePoints, newTp[eqAndInCount], d.rangeMaxSize) + ranges, rangeFallback, err = appendPoints2Ranges(d.sctx, ranges, rangePoints, tmpNewTp, d.rangeMaxSize) } if err != nil { return nil, nil, nil, errors.Trace(err) @@ -533,6 +562,18 @@ func (d *rangeDetacher) buildRangeOnColsByCNFCond(newTp []*types.FieldType, eqAn return ranges, accessConds, nil, nil } +func convertStringFTToBinaryCollate(ft *types.FieldType) *types.FieldType { + if ft.EvalType() != types.ETString || + ft.GetType() == mysql.TypeEnum || + ft.GetType() == mysql.TypeSet { + return ft + } + newTp := ft.Clone() + newTp.SetCharset(charset.CharsetBin) + newTp.SetCollate(charset.CollationBin) + return newTp +} + // buildCNFIndexRange builds the range for index where the top layer is CNF. func (d *rangeDetacher) buildCNFIndexRange(newTp []*types.FieldType, eqAndInCount int, accessConds []expression.Expression) (Ranges, []expression.Expression, []expression.Expression, error) { diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index 48e6378548c68..1fad3ee6e839d 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -1648,21 +1648,21 @@ create table t( exprStr: "f >= 'a' and f <= 'B'", accessConds: "[ge(test.t.f, a) le(test.t.f, B)]", filterConds: "[]", - resultStr: "[[\"a\",\"B\"]]", + resultStr: "[[\"\\x00A\",\"\\x00B\"]]", }, { indexPos: 4, exprStr: "f in ('a', 'B')", accessConds: "[in(test.t.f, a, B)]", filterConds: "[]", - resultStr: "[[\"a\",\"a\"] [\"B\",\"B\"]]", + resultStr: "[[\"\\x00A\",\"\\x00A\"] [\"\\x00B\",\"\\x00B\"]]", }, { indexPos: 4, exprStr: "f = 'a' and f = 'B' collate utf8mb4_bin", accessConds: "[eq(test.t.f, a)]", filterConds: "[eq(test.t.f, B)]", - resultStr: "[[\"a\",\"a\"]]", + resultStr: "[[\"\\x00A\",\"\\x00A\"]]", }, { indexPos: 4,