diff --git a/executor/aggfuncs/builder.go b/executor/aggfuncs/builder.go index d5449d00b35f3..7afde1def0d50 100644 --- a/executor/aggfuncs/builder.go +++ b/executor/aggfuncs/builder.go @@ -51,6 +51,10 @@ func Build(ctx sessionctx.Context, aggFuncDesc *aggregation.AggFuncDesc, ordinal return buildBitXor(aggFuncDesc, ordinal) case ast.AggFuncBitAnd: return buildBitAnd(aggFuncDesc, ordinal) + case ast.AggFuncVarPop: + return buildVarPop(aggFuncDesc, ordinal) + case ast.AggFuncStddevPop: + return buildStdDevPop(aggFuncDesc, ordinal) } return nil } @@ -362,6 +366,48 @@ func buildBitAnd(aggFuncDesc *aggregation.AggFuncDesc, ordinal int) AggFunc { return &bitAndUint64{baseBitAggFunc{base}} } +// buildVarPop builds the AggFunc implementation for function "VAR_POP". +func buildVarPop(aggFuncDesc *aggregation.AggFuncDesc, ordinal int) AggFunc { + base := baseVarPopAggFunc{ + baseAggFunc{ + args: aggFuncDesc.Args, + ordinal: ordinal, + }, + } + switch aggFuncDesc.Mode { + case aggregation.DedupMode: + return nil + default: + if aggFuncDesc.HasDistinct { + return &varPop4DistinctFloat64{base} + } + return &varPop4Float64{base} + } +} + +// buildStdDevPop builds the AggFunc implementation for function "STD()/STDDEV()/STDDEV_POP()" +func buildStdDevPop(aggFuncDesc *aggregation.AggFuncDesc, ordinal int) AggFunc { + base := baseStdDevPopAggFunc{ + varPop4Float64{ + baseVarPopAggFunc{ + baseAggFunc{ + args: aggFuncDesc.Args, + ordinal: ordinal, + }, + }, + }, + } + switch aggFuncDesc.Mode { + case aggregation.DedupMode: + return nil + default: + if aggFuncDesc.HasDistinct { + return &stdDevPop4DistinctFloat64{base} + } + return &stdDevPop4Float64{base} + } +} + // buildRowNumber builds the AggFunc implementation for function "ROW_NUMBER". func buildRowNumber(aggFuncDesc *aggregation.AggFuncDesc, ordinal int) AggFunc { base := baseAggFunc{ diff --git a/executor/aggfuncs/func_stddevpop.go b/executor/aggfuncs/func_stddevpop.go new file mode 100644 index 0000000000000..2db8c06941240 --- /dev/null +++ b/executor/aggfuncs/func_stddevpop.go @@ -0,0 +1,55 @@ +// Copyright 2019 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package aggfuncs + +import ( + "math" + + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/util/chunk" +) + +type baseStdDevPopAggFunc struct { + varPop4Float64 +} + +type stdDevPop4Float64 struct { + baseStdDevPopAggFunc +} + +func (e *stdDevPop4Float64) AppendFinalResult2Chunk(sctx sessionctx.Context, pr PartialResult, chk *chunk.Chunk) error { + p := (*partialResult4VarPopFloat64)(pr) + if p.count == 0 { + chk.AppendNull(e.ordinal) + return nil + } + variance := p.variance / float64(p.count) + chk.AppendFloat64(e.ordinal, math.Sqrt(variance)) + return nil +} + +type stdDevPop4DistinctFloat64 struct { + baseStdDevPopAggFunc +} + +func (e *stdDevPop4DistinctFloat64) AppendFinalResult2Chunk(sctx sessionctx.Context, pr PartialResult, chk *chunk.Chunk) error { + p := (*partialResult4VarPopDistinctFloat64)(pr) + if p.count == 0 { + chk.AppendNull(e.ordinal) + return nil + } + variance := p.variance / float64(p.count) + chk.AppendFloat64(e.ordinal, math.Sqrt(variance)) + return nil +} diff --git a/executor/aggfuncs/func_stddevpop_test.go b/executor/aggfuncs/func_stddevpop_test.go new file mode 100644 index 0000000000000..0c4b7e24f601b --- /dev/null +++ b/executor/aggfuncs/func_stddevpop_test.go @@ -0,0 +1,25 @@ +package aggfuncs_test + +import ( + . "github.com/pingcap/check" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" +) + +func (s *testSuite) TestMergePartialResult4Stddevpop(c *C) { + tests := []aggTest{ + buildAggTester(ast.AggFuncStddevPop, mysql.TypeDouble, 5, 1.4142135623730951, 0.816496580927726, 1.3169567191065923), + } + for _, test := range tests { + s.testMergePartialResult(c, test) + } +} + +func (s *testSuite) TestStddevpop(c *C) { + tests := []aggTest{ + buildAggTester(ast.AggFuncStddevPop, mysql.TypeDouble, 5, nil, 1.4142135623730951), + } + for _, test := range tests { + s.testAggFunc(c, test) + } +} diff --git a/executor/aggfuncs/func_varpop.go b/executor/aggfuncs/func_varpop.go new file mode 100644 index 0000000000000..24c411422b460 --- /dev/null +++ b/executor/aggfuncs/func_varpop.go @@ -0,0 +1,169 @@ +// Copyright 2019 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package aggfuncs + +import ( + "github.com/pingcap/errors" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/set" +) + +type baseVarPopAggFunc struct { + baseAggFunc +} + +type varPop4Float64 struct { + baseVarPopAggFunc +} + +type partialResult4VarPopFloat64 struct { + count int64 + sum float64 + variance float64 +} + +func (e *varPop4Float64) AllocPartialResult() (pr PartialResult) { + return PartialResult(&partialResult4VarPopFloat64{}) +} + +func (e *varPop4Float64) ResetPartialResult(pr PartialResult) { + p := (*partialResult4VarPopFloat64)(pr) + p.count = 0 + p.sum = 0 + p.variance = 0 +} + +func (e *varPop4Float64) AppendFinalResult2Chunk(sctx sessionctx.Context, pr PartialResult, chk *chunk.Chunk) error { + p := (*partialResult4VarPopFloat64)(pr) + if p.count == 0 { + chk.AppendNull(e.ordinal) + return nil + } + variance := p.variance / float64(p.count) + chk.AppendFloat64(e.ordinal, variance) + return nil +} + +func calculateIntermediate(count int64, sum float64, input float64, variance float64) float64 { + t := float64(count)*input - sum + variance += (t * t) / (float64(count * (count - 1))) + return variance +} + +func (e *varPop4Float64) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup []chunk.Row, pr PartialResult) (err error) { + p := (*partialResult4VarPopFloat64)(pr) + for _, row := range rowsInGroup { + input, isNull, err := e.args[0].EvalReal(sctx, row) + if err != nil { + return errors.Trace(err) + } + if isNull { + continue + } + p.count++ + p.sum += input + if p.count > 1 { + p.variance = calculateIntermediate(p.count, p.sum, input, p.variance) + } + } + return nil +} + +func calculateMerge(srcCount, dstCount int64, srcSum, dstSum, srcVariance, dstVariance float64) float64 { + srcCountFloat64 := float64(srcCount) + dstCountFloat64 := float64(dstCount) + + t := (srcCountFloat64/dstCountFloat64)*dstSum - srcSum + dstVariance += srcVariance + ((dstCountFloat64/srcCountFloat64)/(dstCountFloat64+srcCountFloat64))*t*t + return dstVariance +} + +func (e *varPop4Float64) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) (err error) { + p1, p2 := (*partialResult4VarPopFloat64)(src), (*partialResult4VarPopFloat64)(dst) + if p1.count == 0 { + return nil + } + if p2.count == 0 { + p2.count = p1.count + p2.sum = p1.sum + p2.variance = p1.variance + return nil + } + if p2.count != 0 && p1.count != 0 { + p2.variance = calculateMerge(p1.count, p2.count, p1.sum, p2.sum, p1.variance, p2.variance) + p2.count += p1.count + p2.sum += p1.sum + } + return nil +} + +type varPop4DistinctFloat64 struct { + baseVarPopAggFunc +} + +type partialResult4VarPopDistinctFloat64 struct { + count int64 + sum float64 + variance float64 + valSet set.Float64Set +} + +func (e *varPop4DistinctFloat64) AllocPartialResult() (pr PartialResult) { + p := new(partialResult4VarPopDistinctFloat64) + p.count = 0 + p.sum = 0 + p.variance = 0 + p.valSet = set.NewFloat64Set() + return PartialResult(p) +} + +func (e *varPop4DistinctFloat64) ResetPartialResult(pr PartialResult) { + p := (*partialResult4VarPopDistinctFloat64)(pr) + p.count = 0 + p.sum = 0 + p.variance = 0 + p.valSet = set.NewFloat64Set() +} + +func (e *varPop4DistinctFloat64) AppendFinalResult2Chunk(sctx sessionctx.Context, pr PartialResult, chk *chunk.Chunk) error { + p := (*partialResult4VarPopDistinctFloat64)(pr) + if p.count == 0 { + chk.AppendNull(e.ordinal) + return nil + } + variance := p.variance / float64(p.count) + chk.AppendFloat64(e.ordinal, variance) + return nil +} + +func (e *varPop4DistinctFloat64) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup []chunk.Row, pr PartialResult) (err error) { + p := (*partialResult4VarPopDistinctFloat64)(pr) + for _, row := range rowsInGroup { + input, isNull, err := e.args[0].EvalReal(sctx, row) + if err != nil { + return errors.Trace(err) + } + if isNull || p.valSet.Exist(input) { + continue + } + p.valSet.Insert(input) + p.count++ + p.sum += input + if p.count > 1 { + p.variance = calculateIntermediate(p.count, p.sum, input, p.variance) + } + } + return nil +} diff --git a/executor/aggfuncs/func_varpop_test.go b/executor/aggfuncs/func_varpop_test.go new file mode 100644 index 0000000000000..dcd084eda6627 --- /dev/null +++ b/executor/aggfuncs/func_varpop_test.go @@ -0,0 +1,26 @@ +package aggfuncs_test + +import ( + . "github.com/pingcap/check" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/types" +) + +func (s *testSuite) TestMergePartialResult4Varpop(c *C) { + tests := []aggTest{ + buildAggTester(ast.AggFuncVarPop, mysql.TypeDouble, 5, types.NewFloat64Datum(float64(2)), types.NewFloat64Datum(float64(2)/float64(3)), types.NewFloat64Datum(float64(59)/float64(8)-float64(19*19)/float64(8*8))), + } + for _, test := range tests { + s.testMergePartialResult(c, test) + } +} + +func (s *testSuite) TestVarpop(c *C) { + tests := []aggTest{ + buildAggTester(ast.AggFuncVarPop, mysql.TypeDouble, 5, nil, types.NewFloat64Datum(float64(2))), + } + for _, test := range tests { + s.testAggFunc(c, test) + } +} diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index f8dd0ab426bee..1314ba1e38572 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -337,22 +337,96 @@ func (s *testSuite1) TestAggregation(c *C) { tk.MustExec("insert into t value(0), (-0.9871), (-0.9871)") tk.MustQuery("select 10 from t group by a").Check(testkit.Rows("10", "10")) tk.MustQuery("select sum(a) from (select a from t union all select a from t) tmp").Check(testkit.Rows("-3.9484")) - _, err = tk.Exec("select std(a) from t") - c.Assert(errors.Cause(err).Error(), Equals, "unsupported agg function: std") - _, err = tk.Exec("select stddev(a) from t") - c.Assert(errors.Cause(err).Error(), Equals, "unsupported agg function: stddev") - _, err = tk.Exec("select stddev_pop(a) from t") - c.Assert(errors.Cause(err).Error(), Equals, "unsupported agg function: stddev_pop") + + tk.MustExec("drop table t") + tk.MustExec("create table t(a tinyint, b smallint, c mediumint, d int, e bigint, f float, g double, h decimal)") + tk.MustExec("insert into t values(1, 2, 3, 4, 5, 6.1, 7.2, 8.3), (1, 3, 4, 5, 6, 7.1, 8.2, 9.3)") + result = tk.MustQuery("select var_pop(b), var_pop(c), var_pop(d), var_pop(e), var_pop(f), var_pop(g), var_pop(h) from t group by a") + result.Check(testkit.Rows("0.25 0.25 0.25 0.25 0.25 0.25 0.25")) + + tk.MustExec("insert into t values(2, 3, 4, 5, 6, 7.2, 8.3, 9)") + // not support in release-3.0 + //result = tk.MustQuery("select a, var_pop(b) over w, var_pop(c) over w from t window w as (partition by a)").Sort() + //result.Check(testkit.Rows("1 0.25 0.25", "1 0.25 0.25", "2 0 0")) + + tk.MustExec("delete from t where t.a = 2") + tk.MustExec("insert into t values(1, 2, 4, 5, 6, 6.1, 7.2, 9)") + result = tk.MustQuery("select a, var_pop(distinct b), var_pop(distinct c), var_pop(distinct d), var_pop(distinct e), var_pop(distinct f), var_pop(distinct g), var_pop(distinct h) from t group by a") + result.Check(testkit.Rows("1 0.25 0.25 0.25 0.25 0.25 0.25 0.25")) + + tk.MustExec("drop table t") + tk.MustExec("create table t(a int, b bigint, c float, d double, e decimal)") + tk.MustExec("insert into t values(1, 1000, 6.8, 3.45, 8.3), (1, 3998, -3.4, 5.12, 9.3),(1, 288, 9.2, 6.08, 1)") + result = tk.MustQuery("select variance(b), variance(c), variance(d), variance(e) from t group by a") + result.Check(testkit.Rows("2584338.6666666665 29.840000178019228 1.1808222222222229 12.666666666666666")) + + tk.MustExec("insert into t values(1, 255, 6.8, 6.08, 1)") + result = tk.MustQuery("select variance(distinct b), variance(distinct c), variance(distinct d), variance(distinct e) from t group by a") + result.Check(testkit.Rows("2364075.6875 29.840000178019228 1.1808222222222229 12.666666666666666")) + + tk.MustExec("insert into t values(2, 322, 0.8, 2.22, 6)") + // not support in release-3.0 + //result = tk.MustQuery("select a, variance(b) over w from t window w as (partition by a)").Sort() + //result.Check(testkit.Rows("1 2364075.6875", "1 2364075.6875", "1 2364075.6875", "1 2364075.6875", "2 0")) + _, err = tk.Exec("select std_samp(a) from t") // TODO: Fix this error message. c.Assert(errors.Cause(err).Error(), Equals, "[expression:1305]FUNCTION test.std_samp does not exist") - _, err = tk.Exec("select variance(a) from t") - // TODO: Fix this error message. - c.Assert(errors.Cause(err).Error(), Equals, "unsupported agg function: var_pop") - _, err = tk.Exec("select var_pop(a) from t") - c.Assert(errors.Cause(err).Error(), Equals, "unsupported agg function: var_pop") _, err = tk.Exec("select var_samp(a) from t") c.Assert(errors.Cause(err).Error(), Equals, "unsupported agg function: var_samp") + + // For issue #14072: wrong result when using generated column with aggregate statement + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1 (a int, b int generated always as (-a) virtual, c int generated always as (-a) stored);") + tk.MustExec("insert into t1 (a) values (2), (1), (1), (3), (NULL);") + tk.MustQuery("select sum(a) from t1 group by b order by b;").Check(testkit.Rows("", "3", "2", "2")) + tk.MustQuery("select sum(a) from t1 group by c order by c;").Check(testkit.Rows("", "3", "2", "2")) + tk.MustQuery("select sum(b) from t1 group by a order by a;").Check(testkit.Rows("", "-2", "-2", "-3")) + tk.MustQuery("select sum(b) from t1 group by c order by c;").Check(testkit.Rows("", "-3", "-2", "-2")) + tk.MustQuery("select sum(c) from t1 group by a order by a;").Check(testkit.Rows("", "-2", "-2", "-3")) + tk.MustQuery("select sum(c) from t1 group by b order by b;").Check(testkit.Rows("", "-3", "-2", "-2")) + + // For stddev_pop()/std()/stddev() function + tk.MustExec("drop table if exists t1;") + tk.MustExec(`create table t1 (grp int, a bigint unsigned, c char(10) not null);`) + tk.MustExec(`insert into t1 values (1,1,"a");`) + tk.MustExec(`insert into t1 values (2,2,"b");`) + tk.MustExec(`insert into t1 values (2,3,"c");`) + tk.MustExec(`insert into t1 values (3,4,"E");`) + tk.MustExec(`insert into t1 values (3,5,"C");`) + tk.MustExec(`insert into t1 values (3,6,"D");`) + tk.MustQuery(`select stddev_pop(all a) from t1;`).Check(testkit.Rows("1.707825127659933")) + tk.MustQuery(`select stddev_pop(a) from t1 group by grp order by grp;`).Check(testkit.Rows("0", "0.5", "0.816496580927726")) + tk.MustQuery(`select sum(a)+count(a)+avg(a)+stddev_pop(a) as sum from t1 group by grp order by grp;`).Check(testkit.Rows("3", "10", "23.816496580927726")) + tk.MustQuery(`select std(all a) from t1;`).Check(testkit.Rows("1.707825127659933")) + tk.MustQuery(`select std(a) from t1 group by grp order by grp;`).Check(testkit.Rows("0", "0.5", "0.816496580927726")) + tk.MustQuery(`select sum(a)+count(a)+avg(a)+std(a) as sum from t1 group by grp order by grp;`).Check(testkit.Rows("3", "10", "23.816496580927726")) + tk.MustQuery(`select stddev(all a) from t1;`).Check(testkit.Rows("1.707825127659933")) + tk.MustQuery(`select stddev(a) from t1 group by grp order by grp;`).Check(testkit.Rows("0", "0.5", "0.816496580927726")) + tk.MustQuery(`select sum(a)+count(a)+avg(a)+stddev(a) as sum from t1 group by grp order by grp;`).Check(testkit.Rows("3", "10", "23.816496580927726")) + // test null + tk.MustExec("drop table if exists t1;") + tk.MustExec("CREATE TABLE t1 (a int, b int);") + tk.MustQuery("select stddev_pop(b) from t1;").Check(testkit.Rows("")) + tk.MustQuery("select std(b) from t1;").Check(testkit.Rows("")) + tk.MustQuery("select stddev(b) from t1;").Check(testkit.Rows("")) + tk.MustExec("insert into t1 values (1,null);") + tk.MustQuery("select stddev_pop(b) from t1 group by a order by a;").Check(testkit.Rows("")) + tk.MustQuery("select std(b) from t1 group by a order by a;").Check(testkit.Rows("")) + tk.MustQuery("select stddev(b) from t1 group by a order by a;").Check(testkit.Rows("")) + tk.MustExec("insert into t1 values (1,null);") + tk.MustExec("insert into t1 values (2,null);") + tk.MustQuery("select stddev_pop(b) from t1 group by a order by a;").Check(testkit.Rows("", "")) + tk.MustQuery("select std(b) from t1 group by a order by a;").Check(testkit.Rows("", "")) + tk.MustQuery("select stddev(b) from t1 group by a order by a;").Check(testkit.Rows("", "")) + tk.MustExec("insert into t1 values (2,1);") + tk.MustQuery("select stddev_pop(b) from t1 group by a order by a;").Check(testkit.Rows("", "0")) + tk.MustQuery("select std(b) from t1 group by a order by a;").Check(testkit.Rows("", "0")) + tk.MustQuery("select stddev(b) from t1 group by a order by a;").Check(testkit.Rows("", "0")) + tk.MustExec("insert into t1 values (3,1);") + tk.MustQuery("select stddev_pop(b) from t1 group by a order by a;").Check(testkit.Rows("", "0", "0")) + tk.MustQuery("select std(b) from t1 group by a order by a;").Check(testkit.Rows("", "0", "0")) + tk.MustQuery("select stddev(b) from t1 group by a order by a;").Check(testkit.Rows("", "0", "0")) } func (s *testSuite1) TestStreamAggPushDown(c *C) { diff --git a/expression/aggregation/agg_to_pb.go b/expression/aggregation/agg_to_pb.go index a6a003a01a86f..f6ebe7545b0ee 100644 --- a/expression/aggregation/agg_to_pb.go +++ b/expression/aggregation/agg_to_pb.go @@ -52,6 +52,10 @@ func AggFuncToPBExpr(sc *stmtctx.StatementContext, client kv.Client, aggFunc *Ag tp = tipb.ExprType_Agg_BitXor case ast.AggFuncBitAnd: tp = tipb.ExprType_Agg_BitAnd + case ast.AggFuncVarPop: + tp = tipb.ExprType_VarPop + case ast.AggFuncStddevPop: + tp = tipb.ExprType_StddevPop } if !client.IsRequestTypeSupported(kv.ReqTypeSelect, int64(tp)) { return nil diff --git a/expression/aggregation/base_func.go b/expression/aggregation/base_func.go index f578bfcd5c93f..6f483eeb37c23 100644 --- a/expression/aggregation/base_func.go +++ b/expression/aggregation/base_func.go @@ -107,6 +107,10 @@ func (a *baseFuncDesc) typeInfer(ctx sessionctx.Context) error { a.typeInfer4PercentRank() case ast.WindowFuncLead, ast.WindowFuncLag: a.typeInfer4LeadLag(ctx) + case ast.AggFuncVarPop: + a.typeInfer4VarPop(ctx) + case ast.AggFuncStddevPop: + a.typeInfer4Std(ctx) default: return errors.Errorf("unsupported agg function: %s", a.Name) } @@ -235,6 +239,18 @@ func (a *baseFuncDesc) typeInfer4LeadLag(ctx sessionctx.Context) { } } +func (a *baseFuncDesc) typeInfer4VarPop(ctx sessionctx.Context) { + //var_pop's return value type is double + a.RetTp = types.NewFieldType(mysql.TypeDouble) + a.RetTp.Flen, a.RetTp.Decimal = mysql.MaxRealWidth, types.UnspecifiedLength +} + +func (a *baseFuncDesc) typeInfer4Std(ctx sessionctx.Context) { + //std's return value type is double + a.RetTp = types.NewFieldType(mysql.TypeDouble) + a.RetTp.Flen, a.RetTp.Decimal = mysql.MaxRealWidth, types.UnspecifiedLength +} + // GetDefaultValue gets the default value when the function's input is null. // According to MySQL, default values of the function are listed as follows: // e.g. diff --git a/go.mod b/go.mod index 762d6adb6b8db..3745e28278da2 100644 --- a/go.mod +++ b/go.mod @@ -38,7 +38,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e github.com/pingcap/kvproto v0.0.0-20200311073257-e53d835099b0 github.com/pingcap/log v0.0.0-20190715063458-479153f07ebd - github.com/pingcap/parser v3.0.17-0.20200720062348-6dc68ab12230+incompatible + github.com/pingcap/parser v3.0.17-0.20200901060850-21ac2654ca11+incompatible github.com/pingcap/pd v1.1.0-beta.0.20191223090411-ea2b748f6ee2 github.com/pingcap/tidb-tools v3.0.6-0.20191119150227-ff0a3c6e5763+incompatible github.com/pingcap/tipb v0.0.0-20200426072559-d2c068e96eb3 diff --git a/go.sum b/go.sum index bd8734732e2f6..ab384590b7782 100644 --- a/go.sum +++ b/go.sum @@ -158,8 +158,8 @@ github.com/pingcap/kvproto v0.0.0-20200311073257-e53d835099b0 h1:dXXNHvDwAEN1YNg github.com/pingcap/kvproto v0.0.0-20200311073257-e53d835099b0/go.mod h1:QMdbTAXCHzzygQzqcG9uVUgU2fKeSN1GmfMiykdSzzY= github.com/pingcap/log v0.0.0-20190715063458-479153f07ebd h1:hWDol43WY5PGhsh3+8794bFHY1bPrmu6bTalpssCrGg= github.com/pingcap/log v0.0.0-20190715063458-479153f07ebd/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw= -github.com/pingcap/parser v3.0.17-0.20200720062348-6dc68ab12230+incompatible h1:MLxjdAWdBIa9zaQKtDzXl5hXSbXjYcFMeq5F6aUE5Fw= -github.com/pingcap/parser v3.0.17-0.20200720062348-6dc68ab12230+incompatible/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= +github.com/pingcap/parser v3.0.17-0.20200901060850-21ac2654ca11+incompatible h1:pymLtwPxqonEYzYNIZ5CXUKi9gx1AdwrdTlJTU2N5KE= +github.com/pingcap/parser v3.0.17-0.20200901060850-21ac2654ca11+incompatible/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= github.com/pingcap/pd v1.1.0-beta.0.20191223090411-ea2b748f6ee2 h1:NL23b8tsg6M1QpSQedK14/Jx++QeyKL2rGiBvXAQVfA= github.com/pingcap/pd v1.1.0-beta.0.20191223090411-ea2b748f6ee2/go.mod h1:b4gaAPSxaVVtaB+EHamV4Nsv8JmTdjlw0cTKmp4+dRQ= github.com/pingcap/tidb-tools v3.0.6-0.20191119150227-ff0a3c6e5763+incompatible h1:I8HirWsu1MZp6t9G/g8yKCEjJJxtHooKakEgccvdJ4M= diff --git a/planner/core/rule_aggregation_push_down.go b/planner/core/rule_aggregation_push_down.go index 8c31cf555d3e8..4948d02735891 100644 --- a/planner/core/rule_aggregation_push_down.go +++ b/planner/core/rule_aggregation_push_down.go @@ -40,7 +40,7 @@ func (a *aggregationPushDownSolver) isDecomposableWithJoin(fun *aggregation.AggF return false } switch fun.Name { - case ast.AggFuncAvg, ast.AggFuncGroupConcat: + case ast.AggFuncAvg, ast.AggFuncGroupConcat, ast.AggFuncVarPop, ast.AggFuncStddevPop: // TODO: Support avg push down. return false case ast.AggFuncMax, ast.AggFuncMin, ast.AggFuncFirstRow: