Skip to content

Commit

Permalink
Merge branch 'release-4.0' into release-4.0-44830b917465
Browse files Browse the repository at this point in the history
  • Loading branch information
wjhuang2016 authored Nov 17, 2021
2 parents fd162af + fc92ef9 commit a875434
Show file tree
Hide file tree
Showing 132 changed files with 3,950 additions and 1,389 deletions.
151 changes: 82 additions & 69 deletions bindinfo/bind_test.go

Large diffs are not rendered by default.

11 changes: 3 additions & 8 deletions bindinfo/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@ import (

"github.com/pingcap/parser"
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/format"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/metrics"
Expand Down Expand Up @@ -682,14 +681,10 @@ func GenerateBindSQL(ctx context.Context, stmtNode ast.StmtNode, planHint string
// We need to evolve plan based on the current sql, not the original sql which may have different parameters.
// So here we would remove the hint and inject the current best plan hint.
hint.BindHint(stmtNode, &hint.HintsSet{})
var sb strings.Builder
restoreCtx := format.NewRestoreCtx(format.DefaultRestoreFlags, &sb)
restoreCtx.DefaultDB = defaultDB
err := stmtNode.Restore(restoreCtx)
if err != nil {
logutil.Logger(ctx).Debug("[sql-bind] restore SQL failed when generating bind SQL", zap.Error(err))
bindSQL := utilparser.RestoreWithDefaultDB(stmtNode, defaultDB)
if bindSQL == "" {
return ""
}
bindSQL := sb.String()
switch n := stmtNode.(type) {
case *ast.DeleteStmt:
deleteIdx := strings.Index(bindSQL, "DELETE")
Expand Down
359 changes: 180 additions & 179 deletions cmd/explaintest/r/explain_easy.result

Large diffs are not rendered by default.

48 changes: 24 additions & 24 deletions cmd/explaintest/r/explain_easy_stats.result
Original file line number Diff line number Diff line change
Expand Up @@ -102,15 +102,15 @@ id estRows task access object operator info
MemTableScan_4 10000.00 root table:COLUMNS
explain select c2 = (select c2 from t2 where t1.c1 = t2.c1 order by c1 limit 1) from t1;
id estRows task access object operator info
Projection_12 1999.00 root eq(test.t1.c2, test.t2.c2)->Column#8
└─Apply_14 1999.00 root CARTESIAN left outer join
├─TableReader_16(Build) 1999.00 root data:TableFullScan_15
│ └─TableFullScan_15 1999.00 cop[tikv] table:t1 keep order:false
└─Projection_43(Probe) 1.00 root test.t2.c1, test.t2.c2
└─IndexLookUp_42 1.00 root limit embedded(offset:0, count:1)
├─Limit_41(Build) 1.00 cop[tikv] offset:0, count:1
│ └─IndexRangeScan_39 1.25 cop[tikv] table:t2, index:c1(c1) range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true
└─TableRowIDScan_40(Probe) 1.00 cop[tikv] table:t2 keep order:false, stats:pseudo
Projection_13 1999.00 root eq(test.t1.c2, test.t2.c2)->Column#11
└─Apply_15 1999.00 root CARTESIAN left outer join
├─TableReader_17(Build) 1999.00 root data:TableFullScan_16
│ └─TableFullScan_16 1999.00 cop[tikv] table:t1 keep order:false
└─Projection_44(Probe) 1.00 root test.t2.c1, test.t2.c2
└─IndexLookUp_43 1.00 root limit embedded(offset:0, count:1)
├─Limit_42(Build) 1.00 cop[tikv] offset:0, count:1
│ └─IndexRangeScan_40 1.25 cop[tikv] table:t2, index:c1(c1) range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true
└─TableRowIDScan_41(Probe) 1.00 cop[tikv] table:t2 keep order:false, stats:pseudo
explain select * from t1 order by c1 desc limit 1;
id estRows task access object operator info
Limit_10 1.00 root offset:0, count:1
Expand All @@ -120,36 +120,36 @@ Limit_10 1.00 root offset:0, count:1
set @@session.tidb_opt_insubq_to_join_and_agg=0;
explain select 1 in (select c2 from t2) from t1;
id estRows task access object operator info
HashJoin_7 1999.00 root CARTESIAN left outer semi join, other cond:eq(1, test.t2.c2)
├─TableReader_13(Build) 1985.00 root data:TableFullScan_12
│ └─TableFullScan_12 1985.00 cop[tikv] table:t2 keep order:false
└─TableReader_9(Probe) 1999.00 root data:TableFullScan_8
└─TableFullScan_8 1999.00 cop[tikv] table:t1 keep order:false
HashJoin_8 1999.00 root CARTESIAN left outer semi join, other cond:eq(1, test.t2.c2)
├─TableReader_14(Build) 1985.00 root data:TableFullScan_13
│ └─TableFullScan_13 1985.00 cop[tikv] table:t2 keep order:false
└─TableReader_10(Probe) 1999.00 root data:TableFullScan_9
└─TableFullScan_9 1999.00 cop[tikv] table:t1 keep order:false
explain format="dot" select 1 in (select c2 from t2) from t1;
dot contents

digraph HashJoin_7 {
subgraph cluster7{
digraph HashJoin_8 {
subgraph cluster8{
node [style=filled, color=lightgrey]
color=black
label = "root"
"HashJoin_7" -> "TableReader_9"
"HashJoin_7" -> "TableReader_13"
"HashJoin_8" -> "TableReader_10"
"HashJoin_8" -> "TableReader_14"
}
subgraph cluster8{
subgraph cluster9{
node [style=filled, color=lightgrey]
color=black
label = "cop"
"TableFullScan_8"
"TableFullScan_9"
}
subgraph cluster12{
subgraph cluster13{
node [style=filled, color=lightgrey]
color=black
label = "cop"
"TableFullScan_12"
"TableFullScan_13"
}
"TableReader_9" -> "TableFullScan_8"
"TableReader_13" -> "TableFullScan_12"
"TableReader_10" -> "TableFullScan_9"
"TableReader_14" -> "TableFullScan_13"
}

explain select * from index_prune WHERE a = 1010010404050976781 AND b = 26467085526790 LIMIT 1;
Expand Down
66 changes: 33 additions & 33 deletions cmd/explaintest/r/select.result
Original file line number Diff line number Diff line change
Expand Up @@ -382,53 +382,53 @@ drop table if exists t;
create table t(a int, b int);
explain select a != any (select a from t t2) from t t1;
id estRows task access object operator info
Projection_8 10000.00 root and(or(or(gt(Column#8, 1), ne(test.t.a, Column#7)), if(ne(Column#9, 0), <nil>, 0)), and(ne(Column#10, 0), if(isnull(test.t.a), <nil>, 1)))->Column#11
└─HashJoin_9 10000.00 root CARTESIAN inner join
├─StreamAgg_14(Build) 1.00 root funcs:max(Column#13)->Column#7, funcs:count(distinct Column#14)->Column#8, funcs:sum(Column#15)->Column#9, funcs:count(1)->Column#10
│ └─Projection_19 10000.00 root test.t.a, test.t.a, cast(isnull(test.t.a), decimal(22,0) BINARY)->Column#15
│ └─TableReader_18 10000.00 root data:TableFullScan_17
│ └─TableFullScan_17 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo
└─TableReader_12(Probe) 10000.00 root data:TableFullScan_11
└─TableFullScan_11 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo
Projection_9 10000.00 root and(or(or(gt(Column#11, 1), ne(test.t.a, Column#10)), if(ne(Column#12, 0), <nil>, 0)), and(ne(Column#13, 0), if(isnull(test.t.a), <nil>, 1)))->Column#14
└─HashJoin_10 10000.00 root CARTESIAN inner join
├─StreamAgg_15(Build) 1.00 root funcs:max(Column#16)->Column#10, funcs:count(distinct Column#17)->Column#11, funcs:sum(Column#18)->Column#12, funcs:count(1)->Column#13
│ └─Projection_20 10000.00 root test.t.a, test.t.a, cast(isnull(test.t.a), decimal(22,0) BINARY)->Column#18
│ └─TableReader_19 10000.00 root data:TableFullScan_18
│ └─TableFullScan_18 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo
└─TableReader_13(Probe) 10000.00 root data:TableFullScan_12
└─TableFullScan_12 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo
explain select a = all (select a from t t2) from t t1;
id estRows task access object operator info
Projection_8 10000.00 root or(and(and(le(Column#8, 1), eq(test.t.a, Column#7)), if(ne(Column#9, 0), <nil>, 1)), or(eq(Column#10, 0), if(isnull(test.t.a), <nil>, 0)))->Column#11
└─HashJoin_9 10000.00 root CARTESIAN inner join
├─StreamAgg_14(Build) 1.00 root funcs:firstrow(Column#13)->Column#7, funcs:count(distinct Column#14)->Column#8, funcs:sum(Column#15)->Column#9, funcs:count(1)->Column#10
│ └─Projection_19 10000.00 root test.t.a, test.t.a, cast(isnull(test.t.a), decimal(22,0) BINARY)->Column#15
│ └─TableReader_18 10000.00 root data:TableFullScan_17
│ └─TableFullScan_17 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo
└─TableReader_12(Probe) 10000.00 root data:TableFullScan_11
└─TableFullScan_11 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo
Projection_9 10000.00 root or(and(and(le(Column#11, 1), eq(test.t.a, Column#10)), if(ne(Column#12, 0), <nil>, 1)), or(eq(Column#13, 0), if(isnull(test.t.a), <nil>, 0)))->Column#14
└─HashJoin_10 10000.00 root CARTESIAN inner join
├─StreamAgg_15(Build) 1.00 root funcs:firstrow(Column#16)->Column#10, funcs:count(distinct Column#17)->Column#11, funcs:sum(Column#18)->Column#12, funcs:count(1)->Column#13
│ └─Projection_20 10000.00 root test.t.a, test.t.a, cast(isnull(test.t.a), decimal(22,0) BINARY)->Column#18
│ └─TableReader_19 10000.00 root data:TableFullScan_18
│ └─TableFullScan_18 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo
└─TableReader_13(Probe) 10000.00 root data:TableFullScan_12
└─TableFullScan_12 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo
drop table if exists t;
create table t(a int, b int);
drop table if exists s;
create table s(a varchar(20), b varchar(20));
explain select a in (select a from s where s.b = t.b) from t;
id estRows task access object operator info
HashJoin_10 10000.00 root left outer semi join, equal:[eq(Column#8, Column#9)], other cond:eq(cast(test.t.a), cast(test.s.a))
├─Projection_14(Build) 10000.00 root test.s.a, cast(test.s.b, double BINARY)->Column#9
│ └─TableReader_16 10000.00 root data:TableFullScan_15
│ └─TableFullScan_15 10000.00 cop[tikv] table:s keep order:false, stats:pseudo
└─Projection_11(Probe) 10000.00 root test.t.a, cast(test.t.b, double BINARY)->Column#8
└─TableReader_13 10000.00 root data:TableFullScan_12
└─TableFullScan_12 10000.00 cop[tikv] table:t keep order:false, stats:pseudo
HashJoin_11 10000.00 root left outer semi join, equal:[eq(Column#11, Column#12)], other cond:eq(cast(test.t.a), cast(test.s.a))
├─Projection_15(Build) 10000.00 root test.s.a, cast(test.s.b, double BINARY)->Column#12
│ └─TableReader_17 10000.00 root data:TableFullScan_16
│ └─TableFullScan_16 10000.00 cop[tikv] table:s keep order:false, stats:pseudo
└─Projection_12(Probe) 10000.00 root test.t.a, cast(test.t.b, double BINARY)->Column#11
└─TableReader_14 10000.00 root data:TableFullScan_13
└─TableFullScan_13 10000.00 cop[tikv] table:t keep order:false, stats:pseudo
explain select a in (select a+b from t t2 where t2.b = t1.b) from t t1;
id estRows task access object operator info
HashJoin_8 10000.00 root left outer semi join, equal:[eq(test.t.b, test.t.b)], other cond:eq(test.t.a, plus(test.t.a, test.t.b))
├─TableReader_12(Build) 10000.00 root data:TableFullScan_11
│ └─TableFullScan_11 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo
└─TableReader_10(Probe) 10000.00 root data:TableFullScan_9
└─TableFullScan_9 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo
HashJoin_9 10000.00 root left outer semi join, equal:[eq(test.t.b, test.t.b)], other cond:eq(test.t.a, plus(test.t.a, test.t.b))
├─TableReader_13(Build) 10000.00 root data:TableFullScan_12
│ └─TableFullScan_12 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo
└─TableReader_11(Probe) 10000.00 root data:TableFullScan_10
└─TableFullScan_10 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo
drop table t;
create table t(a int not null, b int);
explain select a in (select a from t t2 where t2.b = t1.b) from t t1;
id estRows task access object operator info
HashJoin_8 10000.00 root left outer semi join, equal:[eq(test.t.b, test.t.b) eq(test.t.a, test.t.a)]
├─TableReader_12(Build) 10000.00 root data:TableFullScan_11
│ └─TableFullScan_11 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo
└─TableReader_10(Probe) 10000.00 root data:TableFullScan_9
└─TableFullScan_9 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo
HashJoin_9 10000.00 root left outer semi join, equal:[eq(test.t.b, test.t.b) eq(test.t.a, test.t.a)]
├─TableReader_13(Build) 10000.00 root data:TableFullScan_12
│ └─TableFullScan_12 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo
└─TableReader_11(Probe) 10000.00 root data:TableFullScan_10
└─TableFullScan_10 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo
explain select 1 from (select sleep(1)) t;
id estRows task access object operator info
Projection_4 1.00 root 1->Column#2
Expand Down
20 changes: 10 additions & 10 deletions cmd/explaintest/r/subquery.result
Original file line number Diff line number Diff line change
Expand Up @@ -17,16 +17,16 @@ insert into t values(1,1,1,1),(2,2,2,2),(3,2,2,2),(4,2,2,2),(5,2,2,2);
analyze table t;
explain select t.c in (select count(*) from t s use index(idx), t t1 where s.b = 1 and s.c = 1 and s.d = t.a and s.a = t1.a) from t;
id estRows task access object operator info
Projection_11 5.00 root Column#14
└─Apply_13 5.00 root CARTESIAN left outer semi join, other cond:eq(test.t.c, Column#13)
├─TableReader_15(Build) 5.00 root data:TableFullScan_14
│ └─TableFullScan_14 5.00 cop[tikv] table:t keep order:false
└─StreamAgg_19(Probe) 1.00 root funcs:count(1)->Column#13
└─IndexJoin_22 0.50 root inner join, inner:TableReader_21, outer key:test.t.a, inner key:test.t.a, equal cond:eq(test.t.a, test.t.a)
├─IndexReader_28(Build) 1.00 root index:IndexRangeScan_27
│ └─IndexRangeScan_27 1.00 cop[tikv] table:s, index:idx(b, c, d) range: decided by [eq(test.t.b, 1) eq(test.t.c, 1) eq(test.t.d, test.t.a)], keep order:false
└─TableReader_21(Probe) 1.00 root data:TableRangeScan_20
└─TableRangeScan_20 1.00 cop[tikv] table:t1 range: decided by [test.t.a], keep order:false
Projection_14 5.00 root Column#22
└─Apply_16 5.00 root CARTESIAN left outer semi join, other cond:eq(test.t.c, Column#21)
├─TableReader_18(Build) 5.00 root data:TableFullScan_17
│ └─TableFullScan_17 5.00 cop[tikv] table:t keep order:false
└─StreamAgg_22(Probe) 1.00 root funcs:count(1)->Column#21
└─IndexJoin_25 0.50 root inner join, inner:TableReader_24, outer key:test.t.a, inner key:test.t.a, equal cond:eq(test.t.a, test.t.a)
├─IndexReader_31(Build) 1.00 root index:IndexRangeScan_30
│ └─IndexRangeScan_30 1.00 cop[tikv] table:s, index:idx(b, c, d) range: decided by [eq(test.t.b, 1) eq(test.t.c, 1) eq(test.t.d, test.t.a)], keep order:false
└─TableReader_24(Probe) 1.00 root data:TableRangeScan_23
└─TableRangeScan_23 1.00 cop[tikv] table:t1 range: decided by [test.t.a], keep order:false
drop table if exists t;
create table t(a int, b int, c int);
explain select a from t t1 where t1.a = (select max(t2.a) from t t2 where t1.b=t2.b and t1.c=t2.b);
Expand Down
28 changes: 14 additions & 14 deletions cmd/explaintest/r/tpch.result
Original file line number Diff line number Diff line change
Expand Up @@ -711,20 +711,20 @@ and n_name = 'MOZAMBIQUE'
order by
value desc;
id estRows task access object operator info
Projection_49 1304801.67 root tpch.partsupp.ps_partkey, Column#35
└─Sort_50 1304801.67 root Column#35:desc
└─Selection_52 1304801.67 root gt(Column#35, NULL)
└─HashAgg_53 1631002.09 root group by:Column#44, funcs:sum(Column#42)->Column#35, funcs:firstrow(Column#43)->tpch.partsupp.ps_partkey
└─Projection_73 1631002.09 root mul(tpch.partsupp.ps_supplycost, cast(tpch.partsupp.ps_availqty, decimal(20,0) BINARY))->Column#42, tpch.partsupp.ps_partkey, tpch.partsupp.ps_partkey
└─HashJoin_56 1631002.09 root inner join, equal:[eq(tpch.supplier.s_suppkey, tpch.partsupp.ps_suppkey)]
├─HashJoin_64(Build) 20000.00 root inner join, equal:[eq(tpch.nation.n_nationkey, tpch.supplier.s_nationkey)]
│ ├─TableReader_69(Build) 1.00 root data:Selection_68
│ │ └─Selection_68 1.00 cop[tikv] eq(tpch.nation.n_name, "MOZAMBIQUE")
│ │ └─TableFullScan_67 25.00 cop[tikv] table:nation keep order:false
│ └─TableReader_66(Probe) 500000.00 root data:TableFullScan_65
│ └─TableFullScan_65 500000.00 cop[tikv] table:supplier keep order:false
└─TableReader_71(Probe) 40000000.00 root data:TableFullScan_70
└─TableFullScan_70 40000000.00 cop[tikv] table:partsupp keep order:false
Projection_54 1304801.67 root tpch.partsupp.ps_partkey, Column#35
└─Sort_55 1304801.67 root Column#35:desc
└─Selection_57 1304801.67 root gt(Column#35, NULL)
└─HashAgg_58 1631002.09 root group by:Column#61, funcs:sum(Column#59)->Column#35, funcs:firstrow(Column#60)->tpch.partsupp.ps_partkey
└─Projection_78 1631002.09 root mul(tpch.partsupp.ps_supplycost, cast(tpch.partsupp.ps_availqty, decimal(20,0) BINARY))->Column#59, tpch.partsupp.ps_partkey, tpch.partsupp.ps_partkey
└─HashJoin_61 1631002.09 root inner join, equal:[eq(tpch.supplier.s_suppkey, tpch.partsupp.ps_suppkey)]
├─HashJoin_69(Build) 20000.00 root inner join, equal:[eq(tpch.nation.n_nationkey, tpch.supplier.s_nationkey)]
│ ├─TableReader_74(Build) 1.00 root data:Selection_73
│ │ └─Selection_73 1.00 cop[tikv] eq(tpch.nation.n_name, "MOZAMBIQUE")
│ │ └─TableFullScan_72 25.00 cop[tikv] table:nation keep order:false
│ └─TableReader_71(Probe) 500000.00 root data:TableFullScan_70
│ └─TableFullScan_70 500000.00 cop[tikv] table:supplier keep order:false
└─TableReader_76(Probe) 40000000.00 root data:TableFullScan_75
└─TableFullScan_75 40000000.00 cop[tikv] table:partsupp keep order:false
/*
Q12 Shipping Modes and Order Priority Query
This query determines whether selecting less expensive modes of shipping is negatively affecting the critical-priority
Expand Down
2 changes: 1 addition & 1 deletion config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -687,7 +687,7 @@ var defaultConf = Config{
StmtSummary: StmtSummary{
Enable: true,
EnableInternalQuery: false,
MaxStmtCount: 200,
MaxStmtCount: 3000,
MaxSQLLength: 4096,
RefreshInterval: 1800,
HistorySize: 24,
Expand Down
2 changes: 1 addition & 1 deletion config/config.toml.example
Original file line number Diff line number Diff line change
Expand Up @@ -430,7 +430,7 @@ enable = true
enable-internal-query = false

# max number of statements kept in memory.
max-stmt-count = 200
max-stmt-count = 3000

# max length of displayed normalized sql and sample sql.
max-sql-length = 4096
Expand Down
16 changes: 12 additions & 4 deletions executor/aggfuncs/func_count_distinct.go
Original file line number Diff line number Diff line change
Expand Up @@ -296,6 +296,10 @@ func (e *countOriginalWithDistinct) UpdatePartialResult(sctx sessionctx.Context,
p := (*partialResult4CountWithDistinct)(pr)

encodedBytes := make([]byte, 0)
collators := make([]collate.Collator, 0, len(e.args))
for _, arg := range e.args {
collators = append(collators, collate.GetCollator(arg.GetType().Collate))
}
// Decimal struct is the biggest type we will use.
buf := make([]byte, types.MyDecimalStructSize)

Expand All @@ -304,7 +308,7 @@ func (e *countOriginalWithDistinct) UpdatePartialResult(sctx sessionctx.Context,
encodedBytes = encodedBytes[:0]

for i := 0; i < len(e.args) && !hasNull; i++ {
encodedBytes, isNull, err = evalAndEncode(sctx, e.args[i], row, buf, encodedBytes)
encodedBytes, isNull, err = evalAndEncode(sctx, e.args[i], collators[i], row, buf, encodedBytes)
if err != nil {
return
}
Expand All @@ -325,7 +329,7 @@ func (e *countOriginalWithDistinct) UpdatePartialResult(sctx sessionctx.Context,

// evalAndEncode eval one row with an expression and encode value to bytes.
func evalAndEncode(
sctx sessionctx.Context, arg expression.Expression,
sctx sessionctx.Context, arg expression.Expression, collator collate.Collator,
row chunk.Row, buf, encodedBytes []byte,
) (_ []byte, isNull bool, err error) {
switch tp := arg.GetType().EvalType(); tp {
Expand Down Expand Up @@ -377,7 +381,7 @@ func evalAndEncode(
if err != nil || isNull {
break
}
encodedBytes = codec.EncodeCompactBytes(encodedBytes, hack.Slice(val))
encodedBytes = codec.EncodeCompactBytes(encodedBytes, collator.Key(val))
default:
return nil, false, errors.Errorf("unsupported column type for encode %d", tp)
}
Expand Down Expand Up @@ -756,13 +760,17 @@ func (e *approxCountDistinctOriginal) UpdatePartialResult(sctx sessionctx.Contex
encodedBytes := make([]byte, 0)
// Decimal struct is the biggest type we will use.
buf := make([]byte, types.MyDecimalStructSize)
collators := make([]collate.Collator, 0, len(e.args))
for _, arg := range e.args {
collators = append(collators, collate.GetCollator(arg.GetType().Collate))
}

for _, row := range rowsInGroup {
var hasNull, isNull bool
encodedBytes = encodedBytes[:0]

for i := 0; i < len(e.args) && !hasNull; i++ {
encodedBytes, isNull, err = evalAndEncode(sctx, e.args[i], row, buf, encodedBytes)
encodedBytes, isNull, err = evalAndEncode(sctx, e.args[i], collators[i], row, buf, encodedBytes)
if err != nil {
return
}
Expand Down
Loading

0 comments on commit a875434

Please sign in to comment.