From b7aa9faa574da203ba62515c19be673284e40415 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Fri, 20 May 2022 23:38:38 +0200 Subject: [PATCH] planner: range columns partitioning with explicit collation fix (#32749) (#33742) close pingcap/tidb#32749 --- ddl/db_partition_test.go | 108 +++++++++++++++++++++++ ddl/sanity_check.go | 2 +- expression/collation.go | 2 +- expression/scalar_function.go | 2 +- planner/core/partition_pruner_test.go | 54 ++++++------ planner/core/rule_partition_processor.go | 17 +++- 6 files changed, 154 insertions(+), 31 deletions(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 5d9532b46c..c17c9a6487 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -645,6 +645,114 @@ create table log_message_1 ( tk.MustQuery(`select * from t where a < X'0D' order by a`).Check(testkit.Rows("\x0B", "\x0C")) } +func TestPartitionRangeColumnsCollate(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("create schema PartitionRangeColumnsCollate") + tk.MustExec("use PartitionRangeColumnsCollate") + tk.MustExec(`create table t (a varchar(255) charset utf8mb4 collate utf8mb4_bin) partition by range columns (a) + (partition p0A values less than ("A"), + partition p1AA values less than ("AA"), + partition p2Aa values less than ("Aa"), + partition p3BB values less than ("BB"), + partition p4Bb values less than ("Bb"), + partition p5aA values less than ("aA"), + partition p6aa values less than ("aa"), + partition p7bB values less than ("bB"), + partition p8bb values less than ("bb"), + partition pMax values less than (MAXVALUE))`) + tk.MustExec(`insert into t values ("A"),("a"),("b"),("B"),("aa"),("AA"),("aA"),("Aa"),("BB"),("Bb"),("bB"),("bb"),("AB"),("BA"),("Ab"),("Ba"),("aB"),("bA"),("ab"),("ba")`) + tk.MustQuery(`explain select * from t where a = "AA" collate utf8mb4_general_ci`).Check(testkit.Rows( + `TableReader_7 8000.00 root partition:all data:Selection_6`, + `└─Selection_6 8000.00 cop[tikv] eq(partitionrangecolumnscollate.t.a, "AA")`, + ` └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from t where a = "AA" collate utf8mb4_general_ci`).Sort().Check(testkit.Rows("AA", "Aa", "aA", "aa")) + tk.MustQuery(`explain select * from t where a = "aa" collate utf8mb4_general_ci`).Check(testkit.Rows( + `TableReader_7 8000.00 root partition:all data:Selection_6`, + `└─Selection_6 8000.00 cop[tikv] eq(partitionrangecolumnscollate.t.a, "aa")`, + ` └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from t where a = "aa" collate utf8mb4_general_ci`).Sort().Check(testkit.Rows("AA", "Aa", "aA", "aa")) + tk.MustQuery(`explain select * from t where a >= "aa" collate utf8mb4_general_ci`).Check(testkit.Rows( + `TableReader_7 8000.00 root partition:all data:Selection_6`, + `└─Selection_6 8000.00 cop[tikv] ge(partitionrangecolumnscollate.t.a, "aa")`, + ` └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from t where a >= "aa" collate utf8mb4_general_ci`).Sort().Check(testkit.Rows( + "AA", "AB", "Aa", "Ab", "B", "BA", "BB", "Ba", "Bb", "aA", "aB", "aa", "ab", "b", "bA", "bB", "ba", "bb")) + tk.MustQuery(`explain select * from t where a > "aa" collate utf8mb4_general_ci`).Check(testkit.Rows( + `TableReader_7 8000.00 root partition:all data:Selection_6`, + `└─Selection_6 8000.00 cop[tikv] gt(partitionrangecolumnscollate.t.a, "aa")`, + ` └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from t where a > "aa" collate utf8mb4_general_ci`).Sort().Check(testkit.Rows( + "AB", "Ab", "B", "BA", "BB", "Ba", "Bb", "aB", "ab", "b", "bA", "bB", "ba", "bb")) + tk.MustQuery(`explain select * from t where a <= "aa" collate utf8mb4_general_ci`).Check(testkit.Rows( + `TableReader_7 8000.00 root partition:all data:Selection_6`, + `└─Selection_6 8000.00 cop[tikv] le(partitionrangecolumnscollate.t.a, "aa")`, + ` └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from t where a <= "aa" collate utf8mb4_general_ci`).Sort().Check(testkit.Rows( + "A", "AA", "Aa", "a", "aA", "aa")) + tk.MustQuery(`explain select * from t where a < "aa" collate utf8mb4_general_ci`).Check(testkit.Rows( + `TableReader_7 8000.00 root partition:all data:Selection_6`, + `└─Selection_6 8000.00 cop[tikv] lt(partitionrangecolumnscollate.t.a, "aa")`, + ` └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from t where a < "aa" collate utf8mb4_general_ci`).Sort().Check(testkit.Rows( + "A", "a")) + + tk.MustExec("drop table t") + tk.MustExec(` create table t (a varchar(255) charset utf8mb4 collate utf8mb4_general_ci) partition by range columns (a) +(partition p0 values less than ("A"), + partition p1 values less than ("aa"), + partition p2 values less than ("AAA"), + partition p3 values less than ("aaaa"), + partition p4 values less than ("B"), + partition p5 values less than ("bb"), + partition pMax values less than (MAXVALUE))`) + tk.MustExec(`insert into t values ("A"),("a"),("b"),("B"),("aa"),("AA"),("aA"),("Aa"),("BB"),("Bb"),("bB"),("bb"),("AB"),("BA"),("Ab"),("Ba"),("aB"),("bA"),("ab"),("ba"),("ä"),("ÄÄÄ")`) + tk.MustQuery(`explain select * from t where a = "aa" collate utf8mb4_general_ci`).Check(testkit.Rows( + `TableReader_7 10.00 root partition:p2 data:Selection_6`, + `└─Selection_6 10.00 cop[tikv] eq(partitionrangecolumnscollate.t.a, "aa")`, + ` └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from t where a = "aa" collate utf8mb4_general_ci`).Sort().Check(testkit.Rows( + "AA", "Aa", "aA", "aa")) + tk.MustQuery(`explain select * from t where a = "aa" collate utf8mb4_bin`).Check(testkit.Rows( + `TableReader_7 8000.00 root partition:p2 data:Selection_6`, + `└─Selection_6 8000.00 cop[tikv] eq(partitionrangecolumnscollate.t.a, "aa")`, + ` └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from t where a = "aa" collate utf8mb4_bin`).Sort().Check(testkit.Rows("aa")) + // 'a' < 'b' < 'ä' in _bin + tk.MustQuery(`explain select * from t where a = "ä" collate utf8mb4_bin`).Check(testkit.Rows( + `TableReader_7 8000.00 root partition:p1 data:Selection_6`, + `└─Selection_6 8000.00 cop[tikv] eq(partitionrangecolumnscollate.t.a, "ä")`, + ` └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from t where a = "ä" collate utf8mb4_bin`).Sort().Check(testkit.Rows("ä")) + tk.MustQuery(`explain select * from t where a = "b" collate utf8mb4_bin`).Check(testkit.Rows( + `TableReader_7 8000.00 root partition:p5 data:Selection_6`, + `└─Selection_6 8000.00 cop[tikv] eq(partitionrangecolumnscollate.t.a, "b")`, + ` └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from t where a = "b" collate utf8mb4_bin`).Sort().Check(testkit.Rows("b")) + tk.MustQuery(`explain select * from t where a <= "b" collate utf8mb4_bin`).Check(testkit.Rows( + `TableReader_7 8000.00 root partition:all data:Selection_6`, + `└─Selection_6 8000.00 cop[tikv] le(partitionrangecolumnscollate.t.a, "b")`, + ` └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from t where a <= "b" collate utf8mb4_bin`).Sort().Check(testkit.Rows("A", "AA", "AB", "Aa", "Ab", "B", "BA", "BB", "Ba", "Bb", "a", "aA", "aB", "aa", "ab", "b")) + tk.MustQuery(`explain select * from t where a < "b" collate utf8mb4_bin`).Check(testkit.Rows( + `TableReader_7 8000.00 root partition:all data:Selection_6`, + `└─Selection_6 8000.00 cop[tikv] lt(partitionrangecolumnscollate.t.a, "b")`, + ` └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo`)) + // Missing upper case B if not p5 is included! + tk.MustQuery(`select * from t where a < "b" collate utf8mb4_bin`).Sort().Check(testkit.Rows("A", "AA", "AB", "Aa", "Ab", "B", "BA", "BB", "Ba", "Bb", "a", "aA", "aB", "aa", "ab")) + tk.MustQuery(`explain select * from t where a >= "b" collate utf8mb4_bin`).Check(testkit.Rows( + `TableReader_7 8000.00 root partition:all data:Selection_6`, + `└─Selection_6 8000.00 cop[tikv] ge(partitionrangecolumnscollate.t.a, "b")`, + ` └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from t where a >= "b" collate utf8mb4_bin`).Sort().Check(testkit.Rows("b", "bA", "bB", "ba", "bb", "ÄÄÄ", "ä")) + tk.MustQuery(`explain select * from t where a > "b" collate utf8mb4_bin`).Check(testkit.Rows( + `TableReader_7 8000.00 root partition:all data:Selection_6`, + `└─Selection_6 8000.00 cop[tikv] gt(partitionrangecolumnscollate.t.a, "b")`, + ` └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from t where a > "b" collate utf8mb4_bin`).Sort().Check(testkit.Rows("bA", "bB", "ba", "bb", "ÄÄÄ", "ä")) +} + func TestDisableTablePartition(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() diff --git a/ddl/sanity_check.go b/ddl/sanity_check.go index 528370eeee..ab7b01bd89 100644 --- a/ddl/sanity_check.go +++ b/ddl/sanity_check.go @@ -92,7 +92,7 @@ func (d *ddl) checkDeleteRangeCnt(job *model.Job) { var physicalTableIDs []int64 var ruleIDs []string if err := job.DecodeArgs(&startKey, &physicalTableIDs, &ruleIDs); err != nil { - panic("should not happened") + panic("Error in drop/truncate table, please report a bug with this stack trace and how it happened") } checkRangeCntByTableIDs(physicalTableIDs, cnt) case model.ActionDropTablePartition, model.ActionTruncateTablePartition: diff --git a/expression/collation.go b/expression/collation.go index 2a580ee82e..f24d160157 100644 --- a/expression/collation.go +++ b/expression/collation.go @@ -157,7 +157,7 @@ const ( UNICODE = ASCII | EXTENDED ) -func deriveCoercibilityForScarlarFunc(sf *ScalarFunction) Coercibility { +func deriveCoercibilityForScalarFunc(sf *ScalarFunction) Coercibility { panic("this function should never be called") } diff --git a/expression/scalar_function.go b/expression/scalar_function.go index ee76128faf..d58ed6e579 100644 --- a/expression/scalar_function.go +++ b/expression/scalar_function.go @@ -560,7 +560,7 @@ func (sf *ScalarFunction) GetSingleColumn(reverse bool) (*Column, bool) { // Coercibility returns the coercibility value which is used to check collations. func (sf *ScalarFunction) Coercibility() Coercibility { if !sf.Function.HasCoercibility() { - sf.SetCoercibility(deriveCoercibilityForScarlarFunc(sf)) + sf.SetCoercibility(deriveCoercibilityForScalarFunc(sf)) } return sf.Function.Coercibility() } diff --git a/planner/core/partition_pruner_test.go b/planner/core/partition_pruner_test.go index 5853438545..45f9e1f8f5 100644 --- a/planner/core/partition_pruner_test.go +++ b/planner/core/partition_pruner_test.go @@ -169,7 +169,11 @@ func TestRangeColumnPartitionPruningForInString(t *testing.T) { partitions = append(partitions, strings.Split(parts, ",")...) } } - return strings.Join(partitions, ",") + out := strings.Join(partitions, ",") + if out == "pNull,pAAAA,pCCC,pShrimpsandwich,paaa,pSushi,pMax" { + out = "all" + } + return out } checkColumnStringPruningTests := func(tests []testStruct) { modes := []string{"dynamic", "static"} @@ -177,7 +181,7 @@ func TestRangeColumnPartitionPruningForInString(t *testing.T) { tk.MustExec(`set @@tidb_partition_prune_mode = '` + mode + `'`) for _, test := range tests { explainResult := tk.MustQuery("explain format = 'brief' " + test.sql) - partitions := strings.ToLower(extractPartitions(explainResult)) + partitions := extractPartitions(explainResult) require.Equal(t, test.partitions, partitions, "Mode: %s sql: %s", mode, test.sql) tk.MustQuery(test.sql).Sort().Check(testkit.Rows(test.rows...)) } @@ -193,20 +197,21 @@ func TestRangeColumnPartitionPruningForInString(t *testing.T) { `partition pMax values less than (MAXVALUE))`) tk.MustExec(`insert into t values (NULL), ("a"), ("Räkmacka"), ("🍣 is life"), ("🍺 after work?"), ("🍺🍺🍺🍺🍺 for oktoberfest"),("AA"),("aa"),("AAA"),("aaa")`) tests := []testStruct{ - // Lower case partition names due to issue#32719 - {sql: `select * from t where a IS NULL`, partitions: "pnull", rows: []string{""}}, - {sql: `select * from t where a = 'AA'`, partitions: "paaaa", rows: []string{"AA"}}, - {sql: `select * from t where a = 'AA' collate utf8mb4_general_ci`, partitions: "paaaa", rows: []string{"AA"}}, // Notice that the it not uses _bin collation for partition => 'aa' not found! #32749 + {sql: `select * from t where a IS NULL`, partitions: "pNull", rows: []string{""}}, + {sql: `select * from t where a = 'AA'`, partitions: "pAAAA", rows: []string{"AA"}}, + {sql: `select * from t where a = 'AA' collate utf8mb4_general_ci`, partitions: "all", rows: []string{"AA", "aa"}}, {sql: `select * from t where a = 'aa'`, partitions: "paaa", rows: []string{"aa"}}, - {sql: `select * from t where a = 'aa' collate utf8mb4_general_ci`, partitions: "paaaa", rows: []string{"AA"}}, // Notice that the it not uses _bin collation for partition => 'aa' not found! #32749 - {sql: `select * from t where a = 'AAA'`, partitions: "paaaa", rows: []string{"AAA"}}, - {sql: `select * from t where a = 'AB'`, partitions: "pccc", rows: []string{}}, + {sql: `select * from t where a = 'aa' collate utf8mb4_general_ci`, partitions: "all", rows: []string{"AA", "aa"}}, + {sql: `select * from t where a collate utf8mb4_general_ci = 'aa'`, partitions: "all", rows: []string{"AA", "aa"}}, + {sql: `select * from t where a = 'AAA'`, partitions: "pAAAA", rows: []string{"AAA"}}, + {sql: `select * from t where a = 'AB'`, partitions: "pCCC", rows: []string{}}, {sql: `select * from t where a = 'aB'`, partitions: "paaa", rows: []string{}}, - {sql: `select * from t where a = '🍣'`, partitions: "psushi", rows: []string{}}, - {sql: `select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?")`, partitions: "pshrimpsandwich,psushi,pmax", rows: []string{"Räkmacka", "🍣 is life"}}, - {sql: `select * from t where a in ('AAA', 'aa')`, partitions: "paaaa,paaa", rows: []string{"AAA", "aa"}}, - {sql: `select * from t where a in ('AAA' collate utf8mb4_general_ci, 'aa')`, partitions: "paaaa,paaa", rows: []string{"AA", "AAA", "aa"}}, // aaa missing due to #32749 - {sql: `select * from t where a in ('AAA', 'aa' collate utf8mb4_general_ci)`, partitions: "paaaa", rows: []string{"AA", "AAA"}}, // aa, aaa missing due to #32749 + {sql: `select * from t where a = '🍣'`, partitions: "pSushi", rows: []string{}}, + {sql: `select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?")`, partitions: "pShrimpsandwich,pSushi,pMax", rows: []string{"Räkmacka", "🍣 is life"}}, + {sql: `select * from t where a in ('AAA', 'aa')`, partitions: "pAAAA,paaa", rows: []string{"AAA", "aa"}}, + {sql: `select * from t where a in ('AAA' collate utf8mb4_general_ci, 'aa')`, partitions: "all", rows: []string{"AA", "AAA", "aa", "aaa"}}, + {sql: `select * from t where a in ('AAA', 'aa' collate utf8mb4_general_ci)`, partitions: "all", rows: []string{"AA", "AAA", "aa", "aaa"}}, + {sql: `select * from t where a collate utf8mb4_general_ci in ('AAA', 'aa')`, partitions: "all", rows: []string{"AA", "AAA", "aa", "aaa"}}, } checkColumnStringPruningTests(tests) tk.MustExec(`set names utf8mb4 collate utf8mb4_general_ci`) @@ -225,19 +230,18 @@ func TestRangeColumnPartitionPruningForInString(t *testing.T) { tk.MustExec(`insert into t values (NULL), ("a"), ("Räkmacka"), ("🍣 is life"), ("🍺 after work?"), ("🍺🍺🍺🍺🍺 for oktoberfest"),("AA"),("aa"),("AAA"),("aaa")`) tests = []testStruct{ - // Lower case partition names due to issue#32719 - {sql: `select * from t where a IS NULL`, partitions: "pnull", rows: []string{""}}, + {sql: `select * from t where a IS NULL`, partitions: "pNull", rows: []string{""}}, {sql: `select * from t where a = 'AA'`, partitions: "paaa", rows: []string{"AA", "aa"}}, {sql: `select * from t where a = 'AA' collate utf8mb4_bin`, partitions: "paaa", rows: []string{"AA"}}, - {sql: `select * from t where a = 'AAA'`, partitions: "paaaa", rows: []string{"AAA", "aaa"}}, - {sql: `select * from t where a = 'AAA' collate utf8mb4_bin`, partitions: "paaa", rows: []string{}}, // Notice that the it uses _bin collation for partition => not found! #32749 - {sql: `select * from t where a = 'AB'`, partitions: "pccc", rows: []string{}}, - {sql: `select * from t where a = 'aB'`, partitions: "pccc", rows: []string{}}, - {sql: `select * from t where a = '🍣'`, partitions: "psushi", rows: []string{}}, - {sql: `select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?")`, partitions: "pshrimpsandwich,psushi,pmax", rows: []string{"Räkmacka", "🍣 is life"}}, - {sql: `select * from t where a in ('AA', 'aaa')`, partitions: "paaa,paaaa", rows: []string{"AA", "AAA", "aa", "aaa"}}, - {sql: `select * from t where a in ('AAA' collate utf8mb4_bin, 'aa')`, partitions: "paaa", rows: []string{"aa"}}, // AAA missing due to #32749, why is AA missing? - {sql: `select * from t where a in ('AAA', 'aa' collate utf8mb4_bin)`, partitions: "paaaa,psushi", rows: []string{"AAA"}}, // aa, aaa missing due to #32749 also all missing paaa + {sql: `select * from t where a = 'AAA'`, partitions: "pAAAA", rows: []string{"AAA", "aaa"}}, + {sql: `select * from t where a = 'AAA' collate utf8mb4_bin`, partitions: "pAAAA", rows: []string{"AAA"}}, + {sql: `select * from t where a = 'AB'`, partitions: "pCCC", rows: []string{}}, + {sql: `select * from t where a = 'aB'`, partitions: "pCCC", rows: []string{}}, + {sql: `select * from t where a = '🍣'`, partitions: "pSushi", rows: []string{}}, + {sql: `select * from t where a in ('🍣 is life', "Räkmacka", "🍺🍺🍺🍺 after work?")`, partitions: "pShrimpsandwich,pSushi,pMax", rows: []string{"Räkmacka", "🍣 is life"}}, + {sql: `select * from t where a in ('AA', 'aaa')`, partitions: "paaa,pAAAA", rows: []string{"AA", "AAA", "aa", "aaa"}}, + {sql: `select * from t where a in ('AAA' collate utf8mb4_bin, 'aa')`, partitions: "paaa,pAAAA", rows: []string{"AAA", "aa"}}, + {sql: `select * from t where a in ('AAA', 'aa' collate utf8mb4_bin)`, partitions: "paaa,pAAAA", rows: []string{"AAA", "aa"}}, } tk.MustExec(`set names utf8mb4 collate utf8mb4_bin`) diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 1c48e4e1f9..5747006178 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/mathutil" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/ranger" @@ -1575,13 +1576,23 @@ func (p *rangeColumnsPruner) partitionRangeForExpr(sctx sessionctx.Context, expr return 0, len(p.data), false } - start, end := p.pruneUseBinarySearch(sctx, opName, con, op) + // If different collation, we can only prune if: + // - expression is binary collation (can only be found in one partition) + // - EQ operator, consider values 'a','b','ä' where 'ä' would be in the same partition as 'a' if general_ci, but is binary after 'b' + // otherwise return all partitions / no pruning + _, exprColl := expr.CharsetAndCollation() + colColl := p.partCol.RetType.GetCollate() + if exprColl != colColl && (opName != ast.EQ || !collate.IsBinCollation(exprColl)) { + return 0, len(p.data), true + } + start, end := p.pruneUseBinarySearch(sctx, opName, con) return start, end, true } -func (p *rangeColumnsPruner) pruneUseBinarySearch(sctx sessionctx.Context, op string, data *expression.Constant, f *expression.ScalarFunction) (start int, end int) { +func (p *rangeColumnsPruner) pruneUseBinarySearch(sctx sessionctx.Context, op string, data *expression.Constant) (start int, end int) { var err error var isNull bool + charSet, collation := p.partCol.RetType.GetCharset(), p.partCol.RetType.GetCollate() compare := func(ith int, op string, v *expression.Constant) bool { if ith == len(p.data)-1 { if p.maxvalue { @@ -1590,7 +1601,7 @@ func (p *rangeColumnsPruner) pruneUseBinarySearch(sctx sessionctx.Context, op st } var expr expression.Expression expr, err = expression.NewFunctionBase(sctx, op, types.NewFieldType(mysql.TypeLonglong), p.data[ith], v) - expr.SetCharsetAndCollation(f.CharsetAndCollation()) + expr.SetCharsetAndCollation(charSet, collation) var val int64 val, isNull, err = expr.EvalInt(sctx, chunk.Row{}) return val > 0