-
Notifications
You must be signed in to change notification settings - Fork 5.8k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
executor: support MAX/MIN in new evaluation framework partially #6971
Changes from 8 commits
0904a62
c0bef2d
740c63c
661dcd8
ed6cb9d
208e283
4f2e0a4
93bbe1d
f714f16
b14296c
85a7cda
ab1895a
ec9fa07
6db67e7
cdd3980
70e5c45
b65776e
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,262 @@ | ||
// Copyright 2018 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/juju/errors" | ||
"github.com/pingcap/tidb/sessionctx" | ||
"github.com/pingcap/tidb/types" | ||
"github.com/pingcap/tidb/util/chunk" | ||
) | ||
|
||
type partialResult4MaxMinInt = int64 | ||
type partialResult4MaxMinUint = uint64 | ||
type partialResult4MaxMinDecimal = *types.MyDecimal | ||
type partialResult4MaxMinFloat32 = float32 | ||
type partialResult4MaxMinFloat64 = float64 | ||
|
||
// Todo | ||
type partialResult4MaxMinString string | ||
type partialResult4MaxMinDatetime types.Time | ||
type partialResult4MaxMinTimestamp types.Time | ||
type partialResult4MaxMinDuration types.Duration | ||
|
||
type baseMaxMinAggFunc struct { | ||
baseAggFunc | ||
|
||
isMax bool | ||
// executed is used to indicates: | ||
// 1. whether the partial result is the initialization value which should not be compared during evaluation; | ||
// 2. whether all the values of arg are all null, if so, we should return null as the default value for MAX/MIN. | ||
executed bool | ||
} | ||
|
||
type maxMin4Int struct { | ||
baseMaxMinAggFunc | ||
} | ||
|
||
func (e *maxMin4Int) AllocPartialResult() PartialResult { | ||
return PartialResult(new(partialResult4MaxMinInt)) | ||
} | ||
|
||
func (e *maxMin4Int) ResetPartialResult(pr PartialResult) { | ||
*(*partialResult4MaxMinInt)(pr) = 0 | ||
} | ||
|
||
func (e *maxMin4Int) AppendFinalResult2Chunk(sctx sessionctx.Context, pr PartialResult, chk *chunk.Chunk) error { | ||
if !e.executed { | ||
chk.AppendNull(e.ordinal) | ||
return nil | ||
} | ||
chk.AppendInt64(e.ordinal, *(*partialResult4MaxMinInt)(pr)) | ||
return nil | ||
} | ||
|
||
func (e *maxMin4Int) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup []chunk.Row, pr PartialResult) error { | ||
p := (*partialResult4MaxMinInt)(pr) | ||
for _, row := range rowsInGroup { | ||
input, isNull, err := e.args[0].EvalInt(sctx, row) | ||
if err != nil { | ||
return errors.Trace(err) | ||
} | ||
if isNull { | ||
continue | ||
} | ||
if !e.executed { | ||
*p = input | ||
e.executed = true | ||
continue | ||
} | ||
if e.isMax && input > *p || !e.isMax && input < *p { | ||
*p = input | ||
} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. if e.isMax && input > *p || !e.isMax && input < *p {
*p = input
} |
||
} | ||
return nil | ||
} | ||
|
||
type maxMin4Uint struct { | ||
baseMaxMinAggFunc | ||
} | ||
|
||
func (e *maxMin4Uint) AllocPartialResult() PartialResult { | ||
return PartialResult(new(partialResult4MaxMinUint)) | ||
} | ||
|
||
func (e *maxMin4Uint) ResetPartialResult(pr PartialResult) { | ||
*(*partialResult4MaxMinUint)(pr) = 0 | ||
} | ||
|
||
func (e *maxMin4Uint) AppendFinalResult2Chunk(sctx sessionctx.Context, pr PartialResult, chk *chunk.Chunk) error { | ||
if !e.executed { | ||
chk.AppendNull(e.ordinal) | ||
return nil | ||
} | ||
chk.AppendUint64(e.ordinal, *(*partialResult4MaxMinUint)(pr)) | ||
return nil | ||
} | ||
|
||
func (e *maxMin4Uint) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup []chunk.Row, pr PartialResult) error { | ||
p := (*partialResult4MaxMinUint)(pr) | ||
for _, row := range rowsInGroup { | ||
input, isNull, err := e.args[0].EvalInt(sctx, row) | ||
if err != nil { | ||
return errors.Trace(err) | ||
} | ||
if isNull { | ||
continue | ||
} | ||
i := uint64(input) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. s/i/uintVal/? |
||
if !e.executed { | ||
*p = i | ||
e.executed = true | ||
continue | ||
} | ||
if e.isMax && i > *p || !e.isMax && i < *p { | ||
*p = i | ||
} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. uint64Input := uint64(input)
if !e.executed {
*p = uint64Input
e.executed = true
continue
}
if e.isMax && uint64Input > *p || !e.isMax && uint64Input < *p {
*p = uint64Input
} |
||
} | ||
return nil | ||
} | ||
|
||
// maxMin4Float32 gets a float32 input and returns a float32 result. | ||
type maxMin4Float32 struct { | ||
baseMaxMinAggFunc | ||
} | ||
|
||
func (e *maxMin4Float32) AllocPartialResult() PartialResult { | ||
return PartialResult(new(partialResult4MaxMinFloat32)) | ||
} | ||
|
||
func (e *maxMin4Float32) ResetPartialResult(pr PartialResult) { | ||
*(*partialResult4MaxMinFloat32)(pr) = 0 | ||
} | ||
|
||
func (e *maxMin4Float32) AppendFinalResult2Chunk(sctx sessionctx.Context, pr PartialResult, chk *chunk.Chunk) error { | ||
if !e.executed { | ||
chk.AppendNull(e.ordinal) | ||
return nil | ||
} | ||
chk.AppendFloat32(e.ordinal, *(*partialResult4MaxMinFloat32)(pr)) | ||
return nil | ||
} | ||
|
||
func (e *maxMin4Float32) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup []chunk.Row, pr PartialResult) error { | ||
p := (*partialResult4MaxMinFloat32)(pr) | ||
for _, row := range rowsInGroup { | ||
input, isNull, err := e.args[0].EvalReal(sctx, row) | ||
if err != nil { | ||
return errors.Trace(err) | ||
} | ||
if isNull { | ||
continue | ||
} | ||
f := float32(input) | ||
if !e.executed { | ||
*p = f | ||
e.executed = true | ||
continue | ||
} | ||
if e.isMax && f > *p || !e.isMax && f < *p { | ||
*p = f | ||
} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. float32Input := float32(input)
if !e.executed {
*p = float32Input
e.executed = true
continue
}
if e.isMax && float32Input > *p || !e.isMax && float32Input < *p {
*p = float32Input
} |
||
} | ||
return nil | ||
} | ||
|
||
type maxMin4Float64 struct { | ||
baseMaxMinAggFunc | ||
} | ||
|
||
func (e *maxMin4Float64) AllocPartialResult() PartialResult { | ||
return PartialResult(new(partialResult4MaxMinFloat64)) | ||
} | ||
|
||
func (e *maxMin4Float64) ResetPartialResult(pr PartialResult) { | ||
*(*partialResult4MaxMinFloat64)(pr) = 0 | ||
} | ||
|
||
func (e *maxMin4Float64) AppendFinalResult2Chunk(sctx sessionctx.Context, pr PartialResult, chk *chunk.Chunk) error { | ||
if !e.executed { | ||
chk.AppendNull(e.ordinal) | ||
return nil | ||
} | ||
chk.AppendFloat64(e.ordinal, *(*partialResult4MaxMinFloat64)(pr)) | ||
return nil | ||
} | ||
|
||
func (e *maxMin4Float64) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup []chunk.Row, pr PartialResult) error { | ||
p := (*partialResult4MaxMinFloat64)(pr) | ||
for _, row := range rowsInGroup { | ||
input, isNull, err := e.args[0].EvalReal(sctx, row) | ||
if err != nil { | ||
return errors.Trace(err) | ||
} | ||
if isNull { | ||
continue | ||
} | ||
if !e.executed { | ||
*p = input | ||
e.executed = true | ||
continue | ||
} | ||
if e.isMax && input > *p || !e.isMax && input < *p { | ||
*p = input | ||
} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. if e.isMax && input > *p || !e.isMax && input < *p {
*p = input
} |
||
} | ||
return nil | ||
} | ||
|
||
type maxMin4Decimal struct { | ||
baseMaxMinAggFunc | ||
} | ||
|
||
func (e *maxMin4Decimal) AllocPartialResult() PartialResult { | ||
return PartialResult(new(partialResult4MaxMinDecimal)) | ||
} | ||
|
||
func (e *maxMin4Decimal) ResetPartialResult(pr PartialResult) { | ||
*(*partialResult4MaxMinDecimal)(pr) = partialResult4MaxMinDecimal(*new(partialResult4MaxMinDecimal)) | ||
} | ||
|
||
func (e *maxMin4Decimal) AppendFinalResult2Chunk(sctx sessionctx.Context, pr PartialResult, chk *chunk.Chunk) error { | ||
if !e.executed { | ||
chk.AppendNull(e.ordinal) | ||
return nil | ||
} | ||
chk.AppendMyDecimal(e.ordinal, *(*partialResult4MaxMinDecimal)(pr)) | ||
return nil | ||
} | ||
|
||
func (e *maxMin4Decimal) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup []chunk.Row, pr PartialResult) error { | ||
p := (*partialResult4MaxMinDecimal)(pr) | ||
for _, row := range rowsInGroup { | ||
input, isNull, err := e.args[0].EvalDecimal(sctx, row) | ||
if err != nil { | ||
return errors.Trace(err) | ||
} | ||
if isNull { | ||
continue | ||
} | ||
if !e.executed { | ||
*p = input | ||
e.executed = true | ||
continue | ||
} | ||
cmp := input.Compare(*p) | ||
if e.isMax && cmp == 1 || !e.isMax && cmp == -1 { | ||
*p = input | ||
} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. cmp := input.Compare(*p)
if e.isMax && cmp == 1 || !e.isMax && cmp == -1 {
*p = input
} |
||
} | ||
return nil | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this should be in the field of partial results, and should be reset for every group