Skip to content

Commit

Permalink
executor: buildWindow cannot call typeInfer twice (#30773)
Browse files Browse the repository at this point in the history
close #30402
  • Loading branch information
fzhedu authored Dec 20, 2021
1 parent e1fb2f5 commit e3c56b7
Show file tree
Hide file tree
Showing 4 changed files with 12 additions and 1 deletion.
1 change: 1 addition & 0 deletions executor/aggregate_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1483,6 +1483,7 @@ func TestAvgDecimal(t *testing.T) {
tk.MustExec("insert into td values (0,29815);")
tk.MustExec("insert into td values (10017,-32661);")
tk.MustQuery(" SELECT AVG( col_bigint / col_smallint) AS field1 FROM td;").Sort().Check(testkit.Rows("25769363061037.62077260"))
tk.MustQuery(" SELECT AVG(col_bigint) OVER (PARTITION BY col_smallint) as field2 FROM td where col_smallint = -23828;").Sort().Check(testkit.Rows("4.0000"))
tk.MustExec("drop table td;")
}

Expand Down
2 changes: 1 addition & 1 deletion executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -4192,7 +4192,7 @@ func (b *executorBuilder) buildWindow(v *plannercore.PhysicalWindow) Executor {
partialResults := make([]aggfuncs.PartialResult, 0, len(v.WindowFuncDescs))
resultColIdx := v.Schema().Len() - len(v.WindowFuncDescs)
for _, desc := range v.WindowFuncDescs {
aggDesc, err := aggregation.NewAggFuncDesc(b.ctx, desc.Name, desc.Args, false)
aggDesc, err := aggregation.NewAggFuncDescForWindowFunc(b.ctx, desc, false)
if err != nil {
b.err = err
return nil
Expand Down
1 change: 1 addition & 0 deletions executor/tiflash_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -829,6 +829,7 @@ func (s *tiflashTestSuite) TestAvgOverflow(c *C) {
tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"")
tk.MustExec("set @@session.tidb_enforce_mpp=ON")
tk.MustQuery(" SELECT AVG( col_bigint / col_smallint) AS field1 FROM td;").Sort().Check(testkit.Rows("25769363061037.62077260"))
tk.MustQuery(" SELECT AVG(col_bigint) OVER (PARTITION BY col_smallint) as field2 FROM td where col_smallint = -23828;").Sort().Check(testkit.Rows("4.0000"))
tk.MustExec("drop table if exists td;")
}

Expand Down
9 changes: 9 additions & 0 deletions expression/aggregation/descriptor.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@ type AggFuncDesc struct {
}

// NewAggFuncDesc creates an aggregation function signature descriptor.
// this func cannot be called twice as the TypeInfer has changed the type of args in the first time.
func NewAggFuncDesc(ctx sessionctx.Context, name string, args []expression.Expression, hasDistinct bool) (*AggFuncDesc, error) {
b, err := newBaseFuncDesc(ctx, name, args)
if err != nil {
Expand All @@ -51,6 +52,14 @@ func NewAggFuncDesc(ctx sessionctx.Context, name string, args []expression.Expre
return &AggFuncDesc{baseFuncDesc: b, HasDistinct: hasDistinct}, nil
}

// NewAggFuncDescForWindowFunc creates an aggregation function from window functions, where baseFuncDesc may be ready.
func NewAggFuncDescForWindowFunc(ctx sessionctx.Context, Desc *WindowFuncDesc, hasDistinct bool) (*AggFuncDesc, error) {
if Desc.RetTp == nil { // safety check
return NewAggFuncDesc(ctx, Desc.Name, Desc.Args, hasDistinct)
}
return &AggFuncDesc{baseFuncDesc: baseFuncDesc{Desc.Name, Desc.Args, Desc.RetTp}, HasDistinct: hasDistinct}, nil
}

// String implements the fmt.Stringer interface.
func (a *AggFuncDesc) String() string {
buffer := bytes.NewBufferString(a.Name)
Expand Down

0 comments on commit e3c56b7

Please sign in to comment.